Merge branch '0.9'

Conflicts:
	kafka/codec.py
	kafka/version.py
	test/test_producer.py
	test/test_producer_integration.py
This commit is contained in:
Dana Powers
2016-01-07 18:51:14 -08:00
83 changed files with 7511 additions and 1645 deletions

3
.gitignore vendored
View File

@@ -6,6 +6,7 @@ dist
MANIFEST
env
servers/*/kafka-bin
.coverage
.coverage*
.noseids
docs/_build
.cache*

View File

@@ -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
@@ -48,7 +46,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 `if [ "$TRAVIS_PYTHON_VERSION" == "pypy" ]; then echo pypy; else echo py${TRAVIS_PYTHON_VERSION/./}; fi`
after_success:
- coveralls

View File

@@ -0,0 +1,5 @@
BrokerConnection
================
.. autoclass:: kafka.BrokerConnection
:members:

View File

@@ -0,0 +1,5 @@
KafkaClient
===========
.. autoclass:: kafka.KafkaClient
:members:

View File

@@ -0,0 +1,5 @@
KafkaConsumer
=============
.. autoclass:: kafka.KafkaConsumer
:members:

View File

@@ -0,0 +1,4 @@
KafkaProducer
=============
<unreleased> See :class:`kafka.producer.SimpleProducer`

View File

@@ -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:

View File

@@ -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:

View File

@@ -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:

View File

@@ -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:

View File

@@ -1,7 +1,10 @@
kafka
=====
kafka-python API
****************
.. toctree::
:maxdepth: 4
kafka
KafkaConsumer
KafkaProducer
KafkaClient
BrokerConnection
SimpleProducer

14
docs/compatibility.rst Normal file
View File

@@ -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

View File

@@ -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.
@@ -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

View File

@@ -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
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/
Status
------
The current stable version of this package is `0.9.5 <https://github.com/dpkp/kafka-python/releases/tag/v0.9.5>`_ and is compatible with:
Kafka broker versions
* 0.9.0.0
* 0.8.2.2
* 0.8.2.1
* 0.8.1.1
* 0.8.1
* 0.8.0
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)
License
-------
Apache License, v2.0. See `LICENSE <https://github.com/dpkp/kafka-python/blob/master/LICENSE>`_.
Copyright 2015, David Arthur, Dana Powers, and Contributors
(See `AUTHORS <https://github.com/dpkp/kafka-python/blob/master/AUTHORS.md>`_).
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).
Contents
--------
KafkaConsumer
*************
>>> from kafka import KafkaConsumer
>>> consumer = KafkaConsumer('my_favorite_topic')
>>> for msg in consumer:
... print (msg)
: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.
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.
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.
KafkaProducer
*************
TBD
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.
.. toctree::
:hidden:
:maxdepth: 2
usage
Usage Overview <usage>
API </apidoc/modules>
install
tests
API reference </apidoc/modules>
Indices and tables
==================
* :ref:`genindex`
* :ref:`modindex`
* :ref:`search`
compatibility
support
license

View File

@@ -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

10
docs/license.rst Normal file
View File

@@ -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 <https://github.com/dpkp/kafka-python/blob/master/LICENSE>`_.
Copyright 2016, David Arthur, Dana Powers, and Contributors
(See `AUTHORS <https://github.com/dpkp/kafka-python/blob/master/AUTHORS.md>`_).

11
docs/support.rst Normal file
View File

@@ -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

View File

@@ -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
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
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
------------------
.. code:: bash
To run the tests locally, install tox -- `pip install tox`
See http://tox.readthedocs.org/en/latest/install.html
tox
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
# 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
# see available (pytest) options
tox -e py27 -- --help
Run the integration tests
-------------------------
Integration tests
-----------------
The integration tests will actually start up real local Zookeeper
instance and Kafka brokers, and send messages in using the client.
.. code:: bash
First, get the kafka binaries for integration testing:
KAFKA_VERSION=0.9.0.0 tox -e py27
KAFKA_VERSION=0.8.2.2 tox -e py35
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

View File

@@ -1,46 +1,69 @@
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')
# 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,
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')
for r in responses:
logging.info(r.offset)
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
@@ -49,20 +72,55 @@ Asynchronous Mode
# 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,
producer = SimpleProducer(client,
async=True,
batch_send_every_n=20,
batch_send_every_t=60)
Keyed messages
--------------
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(client,
async=False,
req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE,
ack_timeout=2000,
sync_fail_on_error=False)
responses = producer.send_messages('my-topic', b'another message')
for r in responses:
logging.info(r.offset)
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

View File

@@ -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',
]

View File

@@ -2,19 +2,22 @@ import collections
import copy
import functools
import logging
import select
import random
import time
import six
import kafka.common
from kafka.common import (TopicAndPartition, BrokerMetadata,
from kafka.common import (TopicPartition, 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,
ConnectionStates)
from kafka.protocol import KafkaProtocol
from kafka.util import kafka_bytestring
log = logging.getLogger(__name__)
@@ -31,20 +34,18 @@ 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.topics_to_brokers = {} # TopicPartition -> BrokerMetadata
self.topic_partitions = {} # topic -> partition -> PartitionMetadata
self.load_metadata_for_topics() # bootstrap with all metadata
##################
# Private API #
##################
@@ -52,14 +53,17 @@ 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,
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):
"""
@@ -73,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:
@@ -91,21 +95,21 @@ 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):
"""
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
"""
@@ -129,26 +133,40 @@ 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)
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)
if not conn.connected():
log.warning("Skipping unconnected connection: %s", conn)
continue
request = encoder_fn(payloads=payloads)
future = conn.send(request)
# Block
while not future.is_done:
conn.recv()
if future.failed():
log.error("Request failed: %s", future.exception)
continue
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)
for payload in 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)
decoded = decoder_fn(response)
log.debug('Response %s: %s', requestId, 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')
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):
"""
@@ -178,97 +196,79 @@ 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)
# For each broker, send the list of request payloads
# and collect the responses and errors
broker_failures = []
# For each KafkaConnection 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 payload in payloads:
topic_partition = (payload.topic, payload.partition)
responses[topic_partition] = FailedPayloadsError(payload)
# No exception, try to get response
else:
# 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)
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)
for payload in payloads_by_broker[broker]:
topic_partition = (payload.topic, payload.partition)
responses[topic_partition] = FailedPayloadsError(payload)
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:
refresh_metadata = False
# For each broker, send the list of request payloads
# and collect the responses and errors
payloads_by_broker = self._payloads_by_broker(payloads)
responses = {}
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_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)
future = conn.send(request, expect_response=expect_response)
if future.failed():
refresh_metadata = True
failed_payloads(broker_payloads)
continue
if not expect_response:
for payload in broker_payloads:
topic_partition = (str(payload.topic), payload.partition)
responses[topic_partition] = None
continue
connections_by_future[future] = (conn, broker)
conn = None
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
_, 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()
# Return responses in the same order as provided
@@ -316,7 +316,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:
@@ -387,7 +387,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,14 +398,17 @@ 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():
conn.reinit()
for conn in self._conns.values():
conn.close()
while conn.connect() == ConnectionStates.CONNECTING:
pass
def reset_topic_metadata(self, *topics):
for topic in topics:
@@ -420,14 +423,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 []
@@ -454,89 +455,79 @@ 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.
This method should be called after receiving any error
Updates internal data: broker list, topic/partition list, and
topic/parition -> broker map. 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.
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)
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).
"""
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()
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([(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)
topic_part = TopicPartition(topic, partition)
# Check for partition errors
try:
kafka.common.check_error(partition_metadata)
if error:
error_type = kafka.common.kafka_errors.get(error, UnknownError)
# 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 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:

611
kafka/client_async.py Normal file
View File

@@ -0,0 +1,611 @@
import copy
import heapq
import itertools
import logging
import random
import select
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
from .version import __version__
if six.PY2:
ConnectionError = None
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!
"""
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,
'retry_backoff_ms': 100,
'metadata_max_age_ms': 300000,
}
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
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:
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 = {}
self._connecting = set()
self._delayed_tasks = DelayedTaskQueue()
self._last_bootstrap = 0
self._bootstrap_fails = 0
self._bootstrap(collect_hosts(self.config['bootstrap_servers']))
def _bootstrap(self, hosts):
# Exponential backoff if 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:
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, **self.config)
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 (must be in metadata)"""
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,
**self.config)
return self._finish_connect(node_id)
def _finish_connect(self, 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)
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):
"""Check whether a node is connected and ok to send more requests.
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
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).
Arguments:
node_id (int): the id of the node to close
"""
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 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.
Arguments:
node_id (int): the id of the node to check
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 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
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
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 a request to a specific node.
Arguments:
node_id (int): destination node
request (Struct): request object (not-encoded)
Raises:
NodeNotReadyError: if node_id is not ready
Returns:
Future: resolves to Response struct
"""
if not self._can_send_request(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
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):
"""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']
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, task_future in self._delayed_tasks.pop_ready():
try:
result = task()
except Exception as e:
log.error("Task %s failed: %s", task, e)
task_future.failure(e)
else:
task_future.success(result)
timeout = min(timeout_ms, metadata_timeout,
self.config['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 = 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 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):
"""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
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 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)
inflight = float('inf')
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].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.config['bootstrap_servers']))
return None
def set_topics(self, topics):
"""Set specific topics to track for metadata.
Arguments:
topics (list of str): topics to check for metadata
Returns:
Future: resolves 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.
Returns:
int: milliseconds until next refresh
"""
ttl = self.cluster.ttl()
if ttl > 0:
return ttl
if self._metadata_refresh_in_progress:
return 9999999999
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.
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.
This is a no-op if the task is not scheduled.
Arguments:
task (callable): task to be unscheduled
"""
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
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
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.
Arguments:
task: can be anything, but generally a callable
at (float or int): epoch seconds to schedule task
Returns:
Future: a future that will be returned with the task when ready
"""
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, future)
def next_at(self):
"""Number of seconds until next task is ready."""
self._drop_removed()
if not self._tasks:
return 9999999999
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"""
ready_tasks = []
while self._tasks and self._tasks[0][0] < time.time():
try:
task = self._pop_next()
except KeyError:
break
ready_tasks.append(task)
return ready_tasks

189
kafka/cluster.py Normal file
View File

@@ -0,0 +1,189 @@
from __future__ import absolute_import
import copy
import logging
import random
import time
import kafka.common as Errors
from kafka.common import BrokerMetadata
from .future import Future
log = logging.getLogger(__name__)
class ClusterMetadata(object):
DEFAULT_CONFIG = {
'retry_backoff_ms': 100,
'metadata_max_age_ms': 300000,
}
def __init__(self, **configs):
self._brokers = {}
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.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())
def broker_metadata(self, broker_id):
return self._brokers.get(broker_id)
def partitions_for_topic(self, topic):
if topic not in self._partitions:
return None
return set(self._partitions[topic].keys())
def leader_for_partition(self, partition):
if partition.topic not in self._partitions:
return None
return self._partitions[partition.topic].get(partition.partition)
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:
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):
"""
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
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 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))

View File

@@ -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 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, 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 | | |
+--------+--------------+------------+---------+--------+

View File

@@ -21,37 +21,37 @@ 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("FetchRequestPayload",
["topic", "partition", "offset", "max_bytes"])
FetchResponse = 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
OffsetRequest = namedtuple("OffsetRequest",
OffsetRequestPayload = namedtuple("OffsetRequestPayload",
["topic", "partition", "time", "max_offsets"])
OffsetResponse = 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"])
@@ -72,12 +72,15 @@ OffsetAndMessage = namedtuple("OffsetAndMessage",
Message = namedtuple("Message",
["magic", "attributes", "key", "value"])
TopicAndPartition = namedtuple("TopicAndPartition",
TopicPartition = namedtuple("TopicPartition",
["topic", "partition"])
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",
@@ -90,96 +93,303 @@ RetryOptions = namedtuple("RetryOptions",
class KafkaError(RuntimeError):
retriable = False
# whether metadata should be refreshed on error
invalid_metadata = False
class IllegalStateError(KafkaError):
pass
class IllegalArgumentError(KafkaError):
pass
class NoBrokersAvailable(KafkaError):
retriable = True
invalid_metadata = True
class NodeNotReadyError(KafkaError):
retriable = True
class CorrelationIdError(KafkaError):
retriable = True
class Cancelled(KafkaError):
retriable = True
class TooManyInFlightRequests(KafkaError):
retriable = True
class StaleMetadata(KafkaError):
retriable = True
invalid_metadata = True
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.')
invalid_metadata = True
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.')
retriable = True
invalid_metadata = True
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.')
retriable = True
invalid_metadata = True
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.')
retriable = True
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.')
retriable = True
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.')
retriable = True
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.')
retriable = True
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):
@@ -197,7 +407,8 @@ class FailedPayloadsError(KafkaError):
class ConnectionError(KafkaError):
pass
retriable = True
invalid_metadata = True
class BufferUnderflowError(KafkaError):
@@ -247,6 +458,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

View File

@@ -1,21 +1,347 @@
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
from kafka.common import ConnectionError
import kafka.common as Errors
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__
if six.PY2:
ConnectionError = socket.error
BlockingIOError = Exception
log = logging.getLogger(__name__)
DEFAULT_SOCKET_TIMEOUT_SECONDS = 120
DEFAULT_KAFKA_PORT = 9092
class ConnectionStates(object):
DISCONNECTED = '<disconnected>'
CONNECTING = '<connecting>'
CONNECTED = '<connected>'
InFlightRequest = collections.namedtuple('InFlightRequest',
['request', 'response_type', 'correlation_id', 'future', 'timestamp'])
class BrokerConnection(object):
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,
'api_version': (0, 8, 2), # default to most restrictive
}
def __init__(self, host, port, **configs):
self.host = host
self.port = port
self.in_flight_requests = collections.deque()
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
self._rbuffer = io.BytesIO()
self._receiving = False
self._next_payload_bytes = 0
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.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()
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 to %s returned error %s.'
' Disconnecting.', self, ret)
self.close()
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
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()
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):
"""
Return true if we are disconnected from the given node and can't
re-establish a connection yet
"""
if self.state is ConnectionStates.DISCONNECTED:
backoff = self.config['reconnect_backoff_ms'] / 1000.0
if time.time() < self.last_attempt + backoff:
return True
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
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.ConnectionError()
while self.in_flight_requests:
ifr = self.in_flight_requests.popleft()
ifr.future.failure(error)
def send(self, request, expect_response=True):
"""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.ConnectionError())
if not self.can_send_more():
return future.failure(Errors.TooManyInFlightRequests())
correlation_id = self._next_correlation_id()
header = RequestHeader(request,
correlation_id=correlation_id,
client_id=self.config['client_id'])
message = b''.join([header.encode(), request.encode()])
size = Int32.encode(len(message))
try:
# 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, ConnectionError) as e:
log.exception("Error sending %s to %s", request, self)
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:
ifr = InFlightRequest(request=request,
correlation_id=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 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.
Return response if available
"""
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
# fail all the pending request futures
if self.in_flight_requests:
self.close()
return None
elif not self.in_flight_requests:
log.warning('%s: No in-flight-requests to recv', self)
return None
elif self._requests_timed_out():
log.warning('%s timed out after %s ms. Closing connection.',
self, self.config['request_timeout_ms'])
self.close(error=Errors.RequestTimedOutError(
'Request timed out after %s ms' %
self.config['request_timeout_ms']))
return None
readable, _, _ = select([self._sock], [], [], timeout)
if not readable:
return None
# 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 ConnectionError as e:
if six.PY2 and e.errno == errno.EWOULDBLOCK:
# This shouldn't happen after selecting above
# but just in case
return None
log.exception('%s: Error receiving 4-byte payload header -'
' 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)
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 ConnectionError as e:
# Extremely small chance that we have exactly 4 bytes for a
# header, but nothing to read in the body yet
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:
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):
assert not self._processing, 'Recursion not supported'
self._processing = True
ifr = self.in_flight_requests.popleft()
# verify send/recv correlation ids match
recv_correlation_id = Int32.decode(read_buffer)
# 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))
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)
self._processing = False
return response
def _requests_timed_out(self):
if self.in_flight_requests:
oldest_at = self.in_flight_requests[0].timestamp
timeout = self.config['request_timeout_ms'] / 1000.0
if time.time() >= oldest_at + timeout:
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 "<BrokerConnection host=%s port=%d>" % (self.host, self.port)
def collect_hosts(hosts, randomize=True):
"""
Collects a comma-separated set of hosts (host:port) and optionally
@@ -40,13 +366,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
@@ -79,7 +399,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

View File

@@ -1,6 +1,6 @@
from .simple import SimpleConsumer
from .multiprocess import MultiProcessConsumer
from .kafka import KafkaConsumer
from .group import KafkaConsumer
__all__ = [
'SimpleConsumer', 'MultiProcessConsumer', 'KafkaConsumer'

View File

@@ -7,11 +7,11 @@ from threading import Lock
import kafka.common
from kafka.common import (
OffsetRequest, OffsetCommitRequest, OffsetFetchRequest,
OffsetRequestPayload, OffsetCommitRequestPayload, OffsetFetchRequestPayload,
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 = {}
@@ -94,14 +94,14 @@ 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)
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:
@@ -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
@@ -217,7 +218,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:

645
kafka/consumer/fetcher.py Normal file
View File

@@ -0,0 +1,645 @@
from __future__ import absolute_import
import collections
import copy
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(six.Iterator):
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):
"""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)
for key in self.config:
if key in configs:
self.config[key] = configs[key]
self._client = client
self._subscriptions = subscriptions
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._iterator = None
#self.sensors = FetchManagerMetrics(metrics, metric_group_prefix)
def init_fetches(self):
"""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):
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.
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
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.
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:
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
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)
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):
"""Check FetchResponses for offset out of range.
Raises:
OffsetOutOfRangeError: if any partition from previous FetchResponse
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
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):
"""Check FetchResponses for topic authorization failures.
Raises:
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):
"""Check FetchResponses for messages larger than the max per partition.
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()
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.
Incompatible with iterator interface - use one or the other, not both.
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
AssertionError: if used with iterator (incompatible)
Returns:
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(list)
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
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
# 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 _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 _message_generator(self):
"""Iterate over fetched_records"""
if self._subscriptions.needs_partition_assignment:
raise StopIteration('Subscription needs partition assignment')
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):
# 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:
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
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 __iter__(self): # pylint: disable=non-iterator-returned
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
else:
key = msg.key
if self.config['value_deserializer']:
value = self.config['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.
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:
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.
Arguments:
partition (TopicPartition): The partition that was fetched
future (Future): the future to update based on response
response (OffsetResponse): response from the server
Raises:
AssertionError: if response does not match partition
"""
topic, partition_info = response.topics[0]
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]
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:
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)
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.
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()
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.config['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.config['fetch_max_wait_ms'],
self.config['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 error_type('Unexpected error while fetching data')
"""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)
"""

682
kafka/consumer/group.py Normal file
View File

@@ -0,0 +1,682 @@
from __future__ import absolute_import
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
from kafka.coordinator.consumer import ConsumerCoordinator
from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor
from kafka.protocol.offset import OffsetResetStrategy
from kafka.version import __version__
log = logging.getLogger(__name__)
class KafkaConsumer(six.Iterator):
"""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__,
'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,
'consumer_timeout_ms': -1,
'api_version': 'auto',
'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):
self.config = copy.copy(self.DEFAULT_CONFIG)
for key in self.config:
if key in configs:
self.config[key] = configs.pop(key)
# 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
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)
self._coordinator = ConsumerCoordinator(
self._client, self._subscription,
assignors=self.config['partition_assignment_strategy'],
**self.config)
self._closed = False
self._iterator = None
self._consumer_timeout = float('inf')
#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.
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])
def assignment(self):
"""Get the TopicPartitions currently assigned to this consumer.
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 if the partitions are
in the process of being reassigned).
Returns:
set: {TopicPartition, ...}
"""
return self._subscription.assigned_partitions()
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.config['key_deserializer'].close()
except AttributeError:
pass
try:
self.config['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.
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.
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)
future = self._coordinator.commit_offsets_async(
offsets, callback=callback)
return future
def commit(self, offsets=None):
"""Commit offsets to kafka, blocking until success or error
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.
Blocks until either the commit succeeds or an unrecoverable error is
encountered (in which case it is thrown to the caller).
Currently only supports kafka-topic offset storage (not zookeeper)
Arguments:
offsets (dict, optional): {TopicPartition: OffsetAndMetadata} dict
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)
def committed(self, partition):
"""Get the last committed offset for the given partition
This offset will be used as the position for the consumer
in the event of a failure.
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.
Arguments:
partition (TopicPartition): the partition to check
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:
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:
committed = None
return committed
def topics(self):
"""Get all topic metadata topics the user is authorized to view.
[Not Implemented Yet]
Returns:
{topic: [partition_info]}
"""
raise NotImplementedError('TODO')
def partitions_for_topic(self, topic):
"""Get metadata about the partitions for a given topic.
Arguments:
topic (str): topic to check
Returns:
set: partition ids
"""
return self._client.cluster.partitions_for_topic(topic)
def poll(self, timeout_ms=0):
"""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
records that are available now. Must not be negative. Default: 0
Returns:
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'
assert self._iterator is None, 'Incompatible with iterator interface'
# 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:
return {}
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.
Arguments:
timeout_ms (int): The maximum time in milliseconds to block
Returns:
dict: map of topic to list of records (may be empty)
"""
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()
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
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)
records = self._fetcher.fetched_records()
# 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
self._fetcher.init_fetches()
self._client.poll(timeout_ms / 1000.0)
return self._fetcher.fetched_records()
def position(self, partition):
"""Get the offset of the next record that will be fetched
Arguments:
partition (TopicPartition): partition to check
"""
assert self._subscription.is_assigned(partition)
offset = self._subscription.assignment[partition].consumed
if offset is None:
self._update_fetch_positions(partition)
offset = self._subscription.assignment[partition].consumed
return offset
def pause(self, *partitions):
"""Suspend fetching from the requested partitions.
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:
*partitions (TopicPartition): partitions to pause
"""
for partition in partitions:
log.debug("Pausing partition %s", partition)
self._subscription.pause(partition)
def resume(self, *partitions):
"""Resume fetching from the specified (paused) partitions.
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.
Arguments:
partition (TopicPartition): partition for seek operation
offset (int): message offset in partition
"""
assert offset >= 0
log.debug("Seeking to offset %s for partition %s", offset, partition)
self._subscription.assignment[partition].seek(offset)
def seek_to_beginning(self, *partitions):
"""Seek to the oldest available offset for partitions.
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)
def seek_to_end(self, *partitions):
"""Seek to the most recent available offset for partitions.
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)
def subscribe(self, topics=(), pattern=None, listener=None):
"""Subscribe to a list of topics, or a topic regex pattern
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).
This method is incompatible with assign()
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 topics:
self.unsubscribe()
else:
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:
self._client.set_topics(self._subscription.group_subscription())
log.debug("Subscribed to topic(s): %s", topics)
def subscription(self):
"""Get the current topic subscription.
Returns:
set: {topic, ...}
"""
return self._subscription.subscription
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
"""
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 time.time() < self._consumer_timeout:
if self.config['api_version'] >= (0, 8, 2):
self._coordinator.ensure_coordinator_known()
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
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)
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:
break
def __iter__(self): # pylint: disable=non-iterator-returned
return self
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:
self._iterator = None
raise
# old KafkaConsumer methods are deprecated
def configure(self, **configs):
raise NotImplementedError(
'deprecated -- initialize a new consumer')
def set_topic_partitions(self, *topics):
raise NotImplementedError(
'deprecated -- use subscribe() or assign()')
def fetch_messages(self):
raise NotImplementedError(
'deprecated -- use poll() or iterator interface')
def get_partition_offsets(self, topic, partition,
request_time_ms, max_num_offsets):
raise NotImplementedError(
'deprecated -- send an OffsetRequest with KafkaClient')
def offsets(self, group=None):
raise NotImplementedError('deprecated -- use committed(partition)')
def task_done(self, message):
raise NotImplementedError(
'deprecated -- commit offsets manually if needed')

View File

@@ -9,14 +9,14 @@ import time
import six
from kafka.client import KafkaClient
from kafka import SimpleClient
from kafka.common import (
OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest,
OffsetFetchRequestPayload, OffsetCommitRequestPayload,
OffsetRequestPayload, FetchRequestPayload,
check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError,
OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout,
FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError
)
from kafka.util import kafka_bytestring
logger = logging.getLogger(__name__)
@@ -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)
@@ -192,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:
@@ -212,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):
@@ -230,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
@@ -333,9 +333,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
@@ -353,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)
@@ -425,7 +425,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)
@@ -545,14 +545,14 @@ class KafkaConsumer(object):
continue
commits.append(
OffsetCommitRequest(topic_partition[0], topic_partition[1],
OffsetCommitRequestPayload(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,
self._config['group_id'], commits,
fail_on_error=False
)
@@ -576,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))
@@ -616,8 +615,8 @@ 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']),
[OffsetFetchRequest(topic_partition[0], topic_partition[1])],
self._config['group_id'],
[OffsetFetchRequestPayload(topic_partition[0], topic_partition[1])],
fail_on_error=False)
try:
check_error(resp)
@@ -665,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)

View File

@@ -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,
@@ -104,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

View File

@@ -1,18 +1,15 @@
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
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,
@@ -27,11 +24,12 @@ 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
)
from kafka.protocol.message import PartialMessage
log = logging.getLogger(__name__)
@@ -72,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
@@ -153,9 +151,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
@@ -166,7 +164,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',
@@ -224,23 +222,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 +362,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,
@@ -413,32 +405,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

View File

@@ -0,0 +1,462 @@
from __future__ import absolute_import
import abc
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
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,
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 dynamically assigned via a group coordinator.
Topic subscriptions are not incremental: this list will replace the
current assignment (if there is one).
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)
assert topics or pattern, 'Must provide topics or pattern'
if pattern:
log.info('Subscribing to pattern: /%s/', pattern)
self.subscription = set()
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):
"""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)
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.
Arguments:
topics (list of str): 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):
"""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()
"""
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):
"""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)
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
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()
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).
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():
fetchable.add(partition)
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):
"""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):
"""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):
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):
assert self.has_valid_position, 'Valid consumed/fetch position required'
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):
assert self.has_valid_position, 'Valid consumed/fetch position required'
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
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()
Arguments:
revoked (list of TopicPartition): the 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.
Arguments:
assigned (list of TopicPartition): the partitions assigned to the
consumer (may include partitions that were previously assigned)
"""
pass

View File

@@ -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(

View File

View File

View File

@@ -0,0 +1,54 @@
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
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):
"""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

View File

@@ -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 = 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 = next(member_iter)
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

678
kafka/coordinator/base.py Normal file
View File

@@ -0,0 +1,678 @@
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:
assert future.failed()
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)
"""

View File

@@ -0,0 +1,710 @@
import copy
import collections
import logging
import time
import six
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, 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
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(BaseCoordinator):
"""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, response: True,
'assignors': (),
'session_timeout_ms': 30000,
'heartbeat_interval_ms': 3000,
'retry_backoff_ms': 100,
'api_version': (0, 9),
}
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:
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 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['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,
# 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.config['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():
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():
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.config['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)
assert assignor, '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.config['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)
assert assignor, '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
# 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)
self._client.set_topics(self._subscription.group_subscription())
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
Returns:
bool: True if consumer should rejoin group, 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 for specified partitions
Arguments:
partitions (list of TopicPartition): partitions to fetch
Returns:
dict: {TopicPartition: OffsetAndMetadata}
"""
while True:
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)
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.config['retry_backoff_ms'] / 1000.0)
def close(self):
try:
self._maybe_auto_commit_offsets_sync()
finally:
super(ConsumerCoordinator, self).close()
def commit_offsets_async(self, offsets, callback=None):
"""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
"""
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)
future.add_both(callback, offsets)
def commit_offsets_sync(self, offsets):
"""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
while True:
if self.config['api_version'] >= (0, 8, 2):
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.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
self._auto_commit_task.disable()
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:
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.
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.
Arguments:
offsets (dict of {TopicPartition: OffsetAndMetadata}): what should
be committed
Returns:
Future: indicating whether the commit was successful or not
"""
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)
# create the offset commit request
offset_data = collections.defaultdict(dict)
for tp, offset in six.iteritems(offsets):
offset_data[tp.topic][tp.partition] = offset
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, node_id)
future = Future()
_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
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("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:
unauthorized_topics.add(topic)
elif error_type in (Errors.OffsetMetadataTooLargeError,
Errors.InvalidCommitOffsetSizeError):
# raise the error to the user
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
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):
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_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("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:
log.error("OffsetCommit failed for group % on partition %s"
" with offset %s: %s", self.group_id, tp, offset,
error_type.__name__)
future.failure(error_type())
return
if unauthorized_topics:
log.error("OffsetCommit failed for unauthorized 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.
Arguments:
partitions (list of TopicPartition): the partitions to fetch
Returns:
Future: resolves to dict of offsets: {TopicPartition: int}
"""
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)
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(node_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)
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)
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:
pass
def _reschedule(self, at):
assert self._enabled, 'AutoCommitTask not enabled'
self._client.schedule(self, at)
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")
backoff = self._coordinator.config['retry_backoff_ms'] / 1000.0
self._client.schedule(self, time.time() + backoff)
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);
'''
"""

View File

@@ -0,0 +1,47 @@
import copy
import time
import kafka.common as Errors
class Heartbeat(object):
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]
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
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()

79
kafka/future.py Normal file
View File

@@ -0,0 +1,79 @@
import functools
import logging
import kafka.common as Errors
log = logging.getLogger(__name__)
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):
try:
return self.exception.retriable
except AttributeError:
return False
def success(self, value):
assert not self.is_done, 'Future is already complete'
self.value = value
self.is_done = True
for f in self._callbacks:
try:
f(value)
except Exception:
log.exception('Error processing callback')
return self
def failure(self, e):
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:
f(self.exception)
except Exception:
log.exception('Error processing errback')
return self
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, *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

View File

@@ -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
@@ -15,14 +15,13 @@ from threading import Thread, Event
import six
from kafka.common import (
ProduceRequest, ProduceResponse, TopicAndPartition, RetryOptions,
ProduceRequestPayload, ProduceResponsePayload, TopicPartition, RetryOptions,
kafka_errors, UnsupportedCodecError, FailedPayloadsError,
RequestTimedOutError, AsyncProducerQueueFull, UnknownError,
RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES
)
from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set
from kafka.util import kafka_bytestring
log = logging.getLogger('kafka.producer')
@@ -62,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
@@ -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,
@@ -196,8 +197,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(
@@ -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))
@@ -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,22 +346,37 @@ 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
"""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')`
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.
All messages produced via this method will set the message 'key' to Null
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.
"""
topic = kafka_bytestring(topic)
return self._send_messages(topic, partition, *msg)
def _send_messages(self, topic, partition, *msg, **kwargs):
@@ -380,10 +396,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")
@@ -391,7 +403,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:
@@ -404,7 +416,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,
@@ -449,7 +461,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

View File

@@ -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)

View File

@@ -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

View File

@@ -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)

View File

@@ -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,
)

View File

@@ -0,0 +1,17 @@
import abc
class AbstractType(object):
__metaclass__ = abc.ABCMeta
@abc.abstractmethod
def encode(cls, value): # pylint: disable=no-self-argument
pass
@abc.abstractmethod
def decode(cls, data): # pylint: disable=no-self-argument
pass
@classmethod
def repr(cls, value):
return repr(value)

44
kafka/protocol/admin.py Normal file
View File

@@ -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')))
)

16
kafka/protocol/api.py Normal file
View File

@@ -0,0 +1,16 @@
from .struct import Struct
from .types import Int16, Int32, String, Schema
class RequestHeader(Struct):
SCHEMA = Schema(
('api_key', Int16),
('api_version', Int16),
('correlation_id', Int32),
('client_id', String('utf-8'))
)
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
)

119
kafka/protocol/commit.py Normal file
View File

@@ -0,0 +1,119 @@
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),
('consumer_id', String('utf-8')),
('retention_time', Int64),
('topics', Array(
('topic', String('utf-8')),
('partitions', Array(
('partition', Int32),
('offset', Int64),
('metadata', String('utf-8'))))))
)
DEFAULT_GENERATION_ID = -1
DEFAULT_RETENTION_TIME = -1
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),
('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
RESPONSE_TYPE = OffsetCommitResponse
SCHEMA = Schema(
('consumer_group', String('utf-8')),
('topics', Array(
('topic', String('utf-8')),
('partitions', Array(
('partition', Int32),
('offset', Int64),
('metadata', String('utf-8'))))))
)
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 OffsetFetchRequest_v1(Struct):
API_KEY = 9
API_VERSION = 1 # kafka-backed storage
RESPONSE_TYPE = OffsetFetchResponse
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
RESPONSE_TYPE = OffsetFetchResponse
SCHEMA = Schema(
('consumer_group', String('utf-8')),
('topics', Array(
('topic', String('utf-8')),
('partitions', Array(Int32))))
)
class GroupCoordinatorResponse(Struct):
SCHEMA = Schema(
('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'))
)

32
kafka/protocol/fetch.py Normal file
View File

@@ -0,0 +1,32 @@
from .message import MessageSet
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),
('min_bytes', Int32),
('topics', Array(
('topic', String('utf-8')),
('partitions', Array(
('partition', Int32),
('offset', Int64),
('max_bytes', Int32)))))
)

103
kafka/protocol/group.py Normal file
View File

@@ -0,0 +1,103 @@
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)))
)
UNKNOWN_MEMBER_ID = ''
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'))
)

440
kafka/protocol/legacy.py Normal file
View File

@@ -0,0 +1,440 @@
from __future__ import absolute_import
import logging
import struct
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 (
ProtocolError, ChecksumError,
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
##################
# Public API #
##################
@classmethod
def encode_produce_request(cls, payloads=(), acks=1, timeout=1000):
"""
Encode a ProduceRequest struct
Arguments:
payloads: list of ProduceRequestPayload
acks: How "acky" you want the request to be
1: written to disk by the leader
0: immediate response
-1: waits for all replicas to be in sync
timeout: Maximum time (in ms) the server will wait for replica acks.
This is _not_ a socket timeout
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, response):
"""
Decode ProduceResponse to ProduceResponsePayload
Arguments:
response: ProduceResponse
Return: list of ProduceResponsePayload
"""
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, payloads=(), max_wait_time=100, min_bytes=4096):
"""
Encodes a FetchRequest struct
Arguments:
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, response):
"""
Decode FetchResponse struct to FetchResponsePayloads
Arguments:
response: FetchResponse
"""
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, 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, response):
"""
Decode OffsetResponse into OffsetResponsePayloads
Arguments:
response: OffsetResponse
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, topics=(), payloads=None):
"""
Encode a MetadataRequest
Arguments:
topics: list of strings
"""
if payloads is not None:
topics = payloads
return kafka.protocol.metadata.MetadataRequest(topics)
@classmethod
def decode_metadata_response(cls, response):
return response
@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, group, payloads):
"""
Encode an OffsetCommitRequest struct
Arguments:
group: string, the consumer group you are committing offsets for
payloads: list of OffsetCommitRequestPayload
"""
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))])
@classmethod
def decode_offset_commit_response(cls, response):
"""
Decode OffsetCommitResponse to an OffsetCommitResponsePayload
Arguments:
response: OffsetCommitResponse
"""
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, group, payloads, from_kafka=False):
"""
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:
group: string, the consumer group you are fetching offsets for
payloads: list of OffsetFetchRequestPayload
from_kafka: bool, default False, set True for Kafka-committed offsets
"""
if from_kafka:
request_class = kafka.protocol.commit.OffsetFetchRequest_v1
else:
request_class = kafka.protocol.commit.OffsetFetchRequest_v0
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, response):
"""
Decode OffsetFetchResponse to OffsetFetchResponsePayloads
Arguments:
response: OffsetFetchResponse
"""
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):
"""
Construct a Message
Arguments:
payload: bytes, the payload to send to Kafka
key: bytes, a key used for partition routing (optional)
"""
return kafka.common.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 kafka.common.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 kafka.common.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)

144
kafka/protocol/message.py Normal file
View File

@@ -0,0 +1,144 @@
import io
from ..codec import gzip_decode, snappy_decode
from . import pickle
from .struct import Struct
from .types import (
Int8, Int32, Int64, Bytes, Schema, AbstractType
)
from ..util import crc32
class Message(Struct):
SCHEMA = Schema(
('crc', Int32),
('magic', Int8),
('attributes', Int8),
('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):
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
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:]
@classmethod
def decode(cls, data):
if isinstance(data, bytes):
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])
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
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))
def __hash__(self):
return hash(self._encode_self(recalc_crc=False))
class PartialMessage(bytes):
def __repr__(self):
return 'PartialMessage(%s)' % self
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, 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
# (14 bytes is a message w/ null key and null value)
while bytes_to_read >= 26:
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
# and append a PartialMessage to signal that max_bytes may be too small
if bytes_to_read:
items.append((None, None, PartialMessage(data.read(bytes_to_read))))
return items
@classmethod
def repr(cls, messages):
return '[' + ', '.join([cls.ITEM.repr(m) for m in messages]) + ']'

View File

@@ -0,0 +1,29 @@
from .struct import Struct
from .types import Array, Int16, Int32, Schema, String
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))))))
)
class MetadataRequest(Struct):
API_KEY = 3
API_VERSION = 0
RESPONSE_TYPE = MetadataResponse
SCHEMA = Schema(
('topics', Array(String('utf-8')))
)

36
kafka/protocol/offset.py Normal file
View File

@@ -0,0 +1,36 @@
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(
('topics', Array(
('topic', String('utf-8')),
('partitions', Array(
('partition', Int32),
('error_code', Int16),
('offsets', Array(Int64))))))
)
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),
('time', Int64),
('max_offsets', Int32)))))
)
DEFAULTS = {
'replica_id': -1
}

29
kafka/protocol/pickle.py Normal file
View File

@@ -0,0 +1,29 @@
from __future__ import absolute_import
try:
import copyreg # pylint: disable=import-error
except ImportError:
import copy_reg as copyreg # pylint: disable=import-error
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
copyreg.pickle(types.MethodType, _pickle_method, _unpickle_method)

29
kafka/protocol/produce.py Normal file
View File

@@ -0,0 +1,29 @@
from .message import MessageSet
from .struct import Struct
from .types import Int8, Int16, Int32, Int64, Bytes, String, Array, Schema
class ProduceResponse(Struct):
SCHEMA = Schema(
('topics', Array(
('topic', String('utf-8')),
('partitions', Array(
('partition', Int32),
('error_code', Int16),
('offset', Int64)))))
)
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),
('messages', MessageSet)))))
)

64
kafka/protocol/struct.py Normal file
View File

@@ -0,0 +1,64 @@
#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): # pylint: disable=E0202
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 = []
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) + ')'
def __hash__(self):
return hash(self.encode())
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):
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)
"""

141
kafka/protocol/types.py Normal file
View File

@@ -0,0 +1,141 @@
from __future__ import absolute_import
from struct import pack, unpack
from .abstract import AbstractType
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(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(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(AbstractType):
@classmethod
def encode(cls, value):
return pack('>q', value)
@classmethod
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)
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(AbstractType):
@classmethod
def encode(cls, value):
if value is None:
return Int32.encode(-1)
else:
return Int32.encode(len(value)) + value
@classmethod
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)
def repr(self, value):
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):
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)]
def repr(self, list_of_items):
return '[' + ', '.join([self.array_of.repr(item) for item in list_of_items]) + ']'

View File

@@ -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):
@@ -89,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

View File

@@ -1 +1 @@
__version__ = '0.9.6-dev'
__version__ = '0.97.0-dev'

View File

@@ -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).

View File

@@ -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",

View File

@@ -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

View File

@@ -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

View File

@@ -5,16 +5,18 @@ from mock import ANY, MagicMock, patch
import six
from . import unittest
from kafka import KafkaClient
from kafka import SimpleClient
from kafka.common import (
ProduceRequest, MetadataResponse,
BrokerMetadata, TopicMetadata, PartitionMetadata,
TopicAndPartition, KafkaUnavailableError,
ProduceRequestPayload,
BrokerMetadata,
TopicPartition, KafkaUnavailableError,
LeaderNotAvailableError, UnknownTopicOrPartitionError,
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
from test.testutil import Timer
@@ -22,94 +24,100 @@ NO_ERROR = 0
UNKNOWN_TOPIC_OR_PARTITION = 3
NO_LEADER = 5
class TestKafkaClient(unittest.TestCase):
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 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))
def test_send_broker_unaware_request_fail(self):
'Tests that call fails when all hosts are unavailable'
@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(),
('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)")
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 = SimpleClient(hosts=['kafka01:9092', 'kafka02:9092'])
req = KafkaProtocol.encode_metadata_request(b'client', 0)
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(ANY, '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):
'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[('kafka02', 9092)].recv.return_value = 'valid response'
mocked_conns[('kafka03', 9092)].send.side_effect = RuntimeError("kafka03 went away (unittest)")
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)]
# 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')
with patch.object(SimpleClient, 'load_metadata_for_topics'):
with patch.object(SimpleClient, '_get_conn', side_effect=mock_get_conn):
resp = client._send_broker_unaware_request(payloads=['fake request'],
encoder_fn=MagicMock(),
decoder_fn=lambda x: x)
client = SimpleClient(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.SimpleClient._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),
@@ -117,34 +125,32 @@ 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)
# client loads metadata at init
client = KafkaClient(hosts=['broker_1:4567'])
client = SimpleClient(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]},
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
@@ -156,13 +162,12 @@ 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.KafkaConnection')
@patch('kafka.SimpleClient._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),
@@ -170,16 +175,16 @@ 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)
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'))
@@ -188,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.KafkaConnection')
@patch('kafka.SimpleClient._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),
@@ -200,16 +205,16 @@ 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)
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)
@@ -219,14 +224,13 @@ 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.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"
conn.recv.return_value = 'response' # anything but None
mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -234,18 +238,18 @@ 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)
client = KafkaClient(hosts=['broker_1:4567'])
client = SimpleClient(hosts=['broker_1:4567'])
# topic metadata is loaded but empty
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)
@@ -256,14 +260,14 @@ 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.KafkaConnection')
@patch('kafka.SimpleClient._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),
@@ -271,26 +275,26 @@ 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)
client = KafkaClient(hosts=['broker_1:4567'])
client = SimpleClient(hosts=['broker_1:4567'])
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.KafkaConnection')
@patch('kafka.SimpleClient._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),
@@ -298,20 +302,18 @@ 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'])
client = SimpleClient(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)
@@ -326,21 +328,19 @@ 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.object(SimpleClient, '_get_conn')
@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
mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -348,29 +348,27 @@ 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'])
client = SimpleClient(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.SimpleClient._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),
@@ -378,13 +376,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'])
client = SimpleClient(hosts=['broker_1:4567'])
requests = [ProduceRequest(
requests = [ProduceRequestPayload(
"topic_doesnt_exist", 0,
[create_message("a"), create_message("b")])]
@@ -405,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())

127
test/test_client_async.py Normal file
View File

@@ -0,0 +1,127 @@
import pytest
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
@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)
# host order is randomized internally, so resort before testing
(hosts,), _ = KafkaClient._bootstrap.call_args # pylint: disable=no-member
assert sorted(hosts) == sorted(expected_hosts)
@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
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)])
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():
pass
def test_initiate_connect():
pass
def test_finish_connect():
pass
def test_ready():
pass
def test_close():
pass
def test_is_disconnected():
pass
def test_is_ready():
pass
def test_can_send_request():
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

View File

@@ -1,8 +1,8 @@
import os
from kafka.common import (
FetchRequest, OffsetCommitRequest, OffsetFetchRequest,
KafkaTimeoutError, ProduceRequest
FetchRequestPayload, OffsetCommitRequestPayload, OffsetFetchRequestPayload,
KafkaTimeoutError, ProduceRequestPayload
)
from kafka.protocol import create_message
@@ -28,11 +28,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase):
cls.zk.close()
def test_consume_none(self):
fetch = FetchRequest(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)
@@ -46,25 +46,25 @@ 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)
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 = [
ProduceRequest(
b'foo', 0,
ProduceRequestPayload(
'foo', 0,
[create_message(b'a'), create_message(b'b')]),
ProduceRequest(
b'bar', 1,
ProduceRequestPayload(
'bar', 1,
[create_message(b'a'), create_message(b'b')]),
ProduceRequest(
b'foo', 1,
ProduceRequestPayload(
'foo', 1,
[create_message(b'a'), create_message(b'b')]),
ProduceRequest(
b'bar', 0,
ProduceRequestPayload(
'bar', 0,
[create_message(b'a'), create_message(b'b')]),
]
@@ -82,12 +82,12 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase):
@kafka_versions('>=0.8.1')
def test_commit_fetch_offsets(self):
req = OffsetCommitRequest(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 = OffsetFetchRequest(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

View File

@@ -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"""

View File

@@ -4,7 +4,7 @@ from . import unittest
from kafka import SimpleConsumer, KafkaConsumer, MultiProcessConsumer
from kafka.common import (
KafkaConfigurationError, FetchResponse, OffsetFetchResponse,
KafkaConfigurationError, FetchResponsePayload, OffsetFetchResponsePayload,
FailedPayloadsError, OffsetAndMessage,
NotLeaderForPartitionError, UnknownTopicOrPartitionError
)
@@ -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):
@@ -52,7 +48,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)
@@ -72,7 +68,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)
@@ -86,7 +82,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
@@ -125,11 +121,11 @@ 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,
(OffsetAndMessage(
FetchResponsePayload(payloads[0].topic, payloads[0].partition, 0, 0,
[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))

170
test/test_consumer_group.py Normal file
View File

@@ -0,0 +1,170 @@
import collections
import logging
import threading
import os
import time
import pytest
import six
from kafka import SimpleClient, 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 SimpleClient(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,
heartbeat_interval_ms=500)
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

View File

@@ -7,8 +7,8 @@ from kafka import (
KafkaConsumer, MultiProcessConsumer, SimpleConsumer, create_message
)
from kafka.common import (
ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout,
OffsetOutOfRangeError
ProduceRequestPayload, ConsumerFetchSizeTooSmall,
OffsetOutOfRangeError, TopicPartition
)
from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES
@@ -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
@@ -41,7 +42,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.topic, partition, messages = messages)
resp, = self.client.send_produce_request([produce])
self.assertEqual(resp.error, 0)
@@ -60,10 +61,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]:
@@ -134,7 +136,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
@@ -142,7 +144,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})
def test_simple_consumer__seek(self):
@@ -184,13 +187,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:
@@ -200,7 +204,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)
@@ -304,7 +309,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
@@ -313,6 +318,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})
@@ -369,6 +375,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,
)
@@ -379,6 +387,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,
)
@@ -397,6 +407,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,
)
@@ -414,6 +426,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,
)
@@ -447,11 +461,9 @@ class TestConsumerIntegration(KafkaIntegrationTestCase):
self.send_messages(1, range(100, 200))
# Start a consumer
consumer = self.kafka_consumer(auto_offset_reset='smallest',
consumer_timeout_ms=5000)
consumer = self.kafka_consumer(auto_offset_reset='earliest')
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
@@ -464,13 +476,17 @@ class TestConsumerIntegration(KafkaIntegrationTestCase):
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))
@@ -479,7 +495,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 )
@@ -487,52 +503,48 @@ 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 )
@kafka_versions('>=0.8.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))
# 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',
)
# 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',
)
# 181-200
output_msgs2 = []
with self.assertRaises(ConsumerTimeout):
while True:
m = consumer2.next()
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)

View File

@@ -2,10 +2,11 @@ import logging
import os
import time
from kafka import KafkaClient, SimpleConsumer, KeyedProducer
from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError
from kafka import SimpleClient, SimpleConsumer, KeyedProducer
from kafka.common import (
TopicPartition, FailedPayloadsError, ConnectionError, RequestTimedOutError
)
from kafka.producer.base import Producer
from kafka.util import kafka_bytestring
from test.fixtures import ZookeeperFixture, KafkaFixture
from test.testutil import KafkaIntegrationTestCase, random_string
@@ -31,7 +32,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 = SimpleClient(hosts, timeout=2)
super(TestFailover, self).setUp()
def tearDown(self):
@@ -75,7 +76,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
@@ -160,7 +161,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")
@@ -197,7 +198,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[TopicPartition(topic, partition)]
broker = self.brokers[leader.nodeId]
broker.close()
return broker
@@ -207,7 +208,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,

View File

@@ -2,28 +2,21 @@
import collections
import logging
import threading
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,
ProduceResponse, RetryOptions, TopicAndPartition
ProduceResponsePayload, RetryOptions, TopicPartition
)
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):
@@ -96,12 +89,12 @@ 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, 'ensure_topic_exists'):
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, 'ensure_topic_exists'):
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
@@ -131,7 +124,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
@@ -157,7 +150,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()
@@ -173,7 +166,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))
@@ -187,7 +180,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
@@ -207,7 +200,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]
@@ -227,7 +220,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))
@@ -235,8 +228,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 = []
@@ -244,7 +237,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

View File

@@ -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
@@ -163,6 +163,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase):
producer.stop()
@kafka_versions("all")
def test_producer_random_order(self):
producer = SimpleProducer(self.client, random_start=True)
resp1 = producer.send_messages(self.topic, self.msg("one"), self.msg("two"))
@@ -466,7 +467,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.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.
@@ -484,7 +485,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.topic, partition, start_offset, 1024)])
self.assertEqual(resp.error, 0)
self.assertEqual(resp.partition, partition)

View File

@@ -1,3 +1,4 @@
#pylint: skip-file
from contextlib import contextmanager
import struct
@@ -7,11 +8,11 @@ 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,
BrokerMetadata, TopicMetadata, PartitionMetadata, TopicAndPartition,
OffsetRequestPayload, OffsetCommitRequestPayload, OffsetFetchRequestPayload,
OffsetResponsePayload, OffsetCommitResponsePayload, OffsetFetchResponsePayload,
ProduceRequestPayload, FetchRequestPayload, Message, ChecksumError,
ProduceResponsePayload, FetchResponsePayload, OffsetAndMessage,
BrokerMetadata, TopicMetadata, PartitionMetadata,
KafkaUnavailableError, UnsupportedCodecError, ConsumerFetchSizeTooSmall,
ProtocolError, ConsumerMetadataResponse
)
@@ -173,6 +174,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
@@ -193,6 +195,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 +225,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 +260,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 +281,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'
@@ -296,6 +302,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)
@@ -303,10 +310,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
@@ -329,27 +338,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")
])
]
@@ -398,6 +410,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 +426,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 +467,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"
@@ -470,18 +485,19 @@ 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')
def test_encode_metadata_request_no_topics(self):
expected = b"".join([
struct.pack(">i", 17), # Total length of the request
@@ -496,6 +512,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 +556,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 +606,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 +622,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 +652,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 +677,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 +720,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 +741,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 +777,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
@@ -780,11 +805,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

View File

@@ -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),

View File

@@ -11,9 +11,8 @@ import uuid
from six.moves import xrange
from . import unittest
from kafka import KafkaClient
from kafka.common import OffsetRequest
from kafka.util import kafka_bytestring
from kafka import SimpleClient
from kafka.common import OffsetRequestPayload
__all__ = [
'random_string',
@@ -84,7 +83,6 @@ def get_open_port():
class KafkaIntegrationTestCase(unittest.TestCase):
create_client = True
topic = None
bytes_topic = None
zk = None
server = None
@@ -96,10 +94,9 @@ 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))
self.client = SimpleClient('%s:%d' % (self.server.host, self.server.port))
self.client.ensure_topic_exists(self.topic)
@@ -115,7 +112,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(topic, partition, -1, 1)])
except:
# XXX: We've seen some UnknownErrors here and cant debug w/o server logs
self.zk.child.dump_logs()
@@ -149,6 +146,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)

52
tox.ini
View File

@@ -1,51 +1,39 @@
[tox]
envlist = lint, py26, py27, pypy, py33, py34, py35, docs
envlist = py{26,27,33,34,35,py}, docs
[pytest]
testpaths = kafka test
doctest_optionflags = modules
addopts = --durations=10
[testenv]
deps =
nose
nose-timer
coverage
pytest
pytest-cov
pytest-catchlog
py{27,33,34,35,py}: pytest-pylint
pytest-sugar
pytest-mock
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 --cov=kafka}
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
# pylint doesn't support python2.6
commands = py.test {posargs:--cov=kafka}
[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:pypy]
# pylint is super slow on pypy...
commands = py.test {posargs:--cov=kafka}
[testenv:docs]
deps =
sphinxcontrib-napoleon
sphinx_rtd_theme
sphinx

View File

@@ -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;