diff --git a/.gitignore b/.gitignore index edb75c547..f3cd082fa 100644 --- a/.gitignore +++ b/.gitignore @@ -13,4 +13,5 @@ docs/_build .cache* .idea/ integration-test/ -tests-env/ \ No newline at end of file +tests-env/ +.pytest_cache/ diff --git a/.travis.yml b/.travis.yml index 91bc3af4a..9758988cf 100644 --- a/.travis.yml +++ b/.travis.yml @@ -12,6 +12,7 @@ env: - KAFKA_VERSION=0.9.0.1 - KAFKA_VERSION=0.10.2.1 - KAFKA_VERSION=0.11.0.2 + - KAFKA_VERSION=1.0.1 sudo: false diff --git a/CHANGES.md b/CHANGES.md index e9d1e879b..288ae9095 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,182 @@ +# 1.4.3 (May 26, 2018) + +Compatibility +* Fix for python 3.7 support: remove 'async' keyword from SimpleProducer (dpkp #1454) + +Client +* Improve BrokerConnection initialization time (romulorosa #1475) +* Ignore MetadataResponses with empty broker list (dpkp #1506) +* Improve connection handling when bootstrap list is invalid (dpkp #1507) + +Consumer +* Check for immediate failure when looking up coordinator in heartbeat thread (dpkp #1457) + +Core / Protocol +* Always acquire client lock before coordinator lock to avoid deadlocks (dpkp #1464) +* Added AlterConfigs and DescribeConfigs apis (StephenSorriaux #1472) +* Fix CreatePartitionsRequest_v0 (StephenSorriaux #1469) +* Add codec validators to record parser and builder for all formats (tvoinarovskyi #1447) +* Fix MemoryRecord bugs re error handling and add test coverage (tvoinarovskyi #1448) +* Force lz4 to disable Kafka-unsupported block linking when encoding (mnito #1476) +* Stop shadowing `ConnectionError` (jeffwidman #1492) + +Documentation +* Document methods that return None (jeffwidman #1504) +* Minor doc capitalization cleanup (jeffwidman) +* Adds add_callback/add_errback example to docs (Berkodev #1441) +* Fix KafkaConsumer docstring for request_timeout_ms default (dpkp #1459) + +Test Infrastructure +* Skip flakey SimpleProducer test (dpkp) +* Fix skipped integration tests if KAFKA_VERSION unset (dpkp #1453) + +Logging / Error Messages +* Stop using deprecated log.warn() (jeffwidman) +* Change levels for some heartbeat thread logging (dpkp #1456) +* Log Heartbeat thread start / close for debugging (dpkp) + + +# 1.4.2 (Mar 10, 2018) + +Bugfixes +* Close leaked selector in version check (dpkp #1425) +* Fix `BrokerConnection.connection_delay()` to return milliseconds (dpkp #1414) +* Use local copies in `Fetcher._fetchable_partitions` to avoid mutation errors (dpkp #1400) +* Fix error var name in `_unpack` (j2gg0s #1403) +* Fix KafkaConsumer compacted offset handling (dpkp #1397) +* Fix byte size estimation with kafka producer (blakeembrey #1393) +* Fix coordinator timeout in consumer poll interface (braedon #1384) + +Client +* Add `BrokerConnection.connect_blocking()` to improve bootstrap to multi-address hostnames (dpkp #1411) +* Short-circuit `BrokerConnection.close()` if already disconnected (dpkp #1424) +* Only increase reconnect backoff if all addrinfos have been tried (dpkp #1423) +* Make BrokerConnection .host / .port / .afi immutable to avoid incorrect 'metadata changed' checks (dpkp #1422) +* Connect with sockaddrs to support non-zero ipv6 scope ids (dpkp #1433) +* Check timeout type in KafkaClient constructor (asdaraujo #1293) +* Update string representation of SimpleClient (asdaraujo #1293) +* Do not validate `api_version` against known versions (dpkp #1434) + +Consumer +* Avoid tight poll loop in consumer when brokers are down (dpkp #1415) +* Validate `max_records` in KafkaConsumer.poll (dpkp #1398) +* KAFKA-5512: Awake heartbeat thread when it is time to poll (dpkp #1439) + +Producer +* Validate that serializers generate bytes-like (or None) data (dpkp #1420) + +Core / Protocol +* Support alternative lz4 package: lz4framed (everpcpc #1395) +* Use hardware accelerated CRC32C function if available (tvoinarovskyi #1389) +* Add Admin CreatePartitions API call (alexef #1386) + +Test Infrastructure +* Close KafkaConsumer instances during tests (dpkp #1410) +* Introduce new fixtures to prepare for migration to pytest (asdaraujo #1293) +* Removed pytest-catchlog dependency (asdaraujo #1380) +* Fixes racing condition when message is sent to broker before topic logs are created (asdaraujo #1293) +* Add kafka 1.0.1 release to test fixtures (dpkp #1437) + +Logging / Error Messages +* Re-enable logging during broker version check (dpkp #1430) +* Connection logging cleanups (dpkp #1432) +* Remove old CommitFailed error message from coordinator (dpkp #1436) + + +# 1.4.1 (Feb 9, 2018) + +Bugfixes +* Fix consumer poll stuck error when no available partition (ckyoog #1375) +* Increase some integration test timeouts (dpkp #1374) +* Use raw in case string overriden (jeffwidman #1373) +* Fix pending completion IndexError bug caused by multiple threads (dpkp #1372) + + +# 1.4.0 (Feb 6, 2018) + +This is a substantial release. Although there are no known 'showstopper' bugs as of release, +we do recommend you test any planned upgrade to your application prior to running in production. + +Some of the major changes include: +* We have officially dropped python 2.6 support +* The KafkaConsumer now includes a background thread to handle coordinator heartbeats +* API protocol handling has been separated from networking code into a new class, KafkaProtocol +* Added support for kafka message format v2 +* Refactored DNS lookups during kafka broker connections +* SASL authentication is working (we think) +* Removed several circular references to improve gc on close() + +Thanks to all contributors -- the state of the kafka-python community is strong! + +Detailed changelog are listed below: + +Client +* Fixes for SASL support + * Refactor SASL/gssapi support (dpkp #1248 #1249 #1257 #1262 #1280) + * Add security layer negotiation to the GSSAPI authentication (asdaraujo #1283) + * Fix overriding sasl_kerberos_service_name in KafkaConsumer / KafkaProducer (natedogs911 #1264) + * Fix typo in _try_authenticate_plain (everpcpc #1333) + * Fix for Python 3 byte string handling in SASL auth (christophelec #1353) +* Move callback processing from BrokerConnection to KafkaClient (dpkp #1258) +* Use socket timeout of request_timeout_ms to prevent blocking forever on send (dpkp #1281) +* Refactor dns lookup in BrokerConnection (dpkp #1312) +* Read all available socket bytes (dpkp #1332) +* Honor reconnect_backoff in conn.connect() (dpkp #1342) + +Consumer +* KAFKA-3977: Defer fetch parsing for space efficiency, and to raise exceptions to user (dpkp #1245) +* KAFKA-4034: Avoid unnecessary consumer coordinator lookup (dpkp #1254) +* Handle lookup_coordinator send failures (dpkp #1279) +* KAFKA-3888 Use background thread to process consumer heartbeats (dpkp #1266) +* Improve KafkaConsumer cleanup (dpkp #1339) +* Fix coordinator join_future race condition (dpkp #1338) +* Avoid KeyError when filtering fetchable partitions (dpkp #1344) +* Name heartbeat thread with group_id; use backoff when polling (dpkp #1345) +* KAFKA-3949: Avoid race condition when subscription changes during rebalance (dpkp #1364) +* Fix #1239 regression to avoid consuming duplicate compressed messages from mid-batch (dpkp #1367) + +Producer +* Fix timestamp not passed to RecordMetadata (tvoinarovskyi #1273) +* Raise non-API exceptions (jeffwidman #1316) +* Fix reconnect_backoff_max_ms default config bug in KafkaProducer (YaoC #1352) + +Core / Protocol +* Add kafka.protocol.parser.KafkaProtocol w/ receive and send (dpkp #1230) +* Refactor MessageSet and Message into LegacyRecordBatch to later support v2 message format (tvoinarovskyi #1252) +* Add DefaultRecordBatch implementation aka V2 message format parser/builder. (tvoinarovskyi #1185) +* optimize util.crc32 (ofek #1304) +* Raise better struct pack/unpack errors (jeffwidman #1320) +* Add Request/Response structs for kafka broker 1.0.0 (dpkp #1368) + +Bugfixes +* use python standard max value (lukekingbru #1303) +* changed for to use enumerate() (TheAtomicOption #1301) +* Explicitly check for None rather than falsey (jeffwidman #1269) +* Minor Exception cleanup (jeffwidman #1317) +* Use non-deprecated exception handling (jeffwidman a699f6a) +* Remove assertion with side effect in client.wakeup() (bgedik #1348) +* use absolute imports everywhere (kevinkjt2000 #1362) + +Test Infrastructure +* Use 0.11.0.2 kafka broker for integration testing (dpkp #1357 #1244) +* Add a Makefile to help build the project, generate docs, and run tests (tvoinarovskyi #1247) +* Add fixture support for 1.0.0 broker (dpkp #1275) +* Add kafka 1.0.0 to travis integration tests (dpkp #1365) +* Change fixture default host to localhost (asdaraujo #1305) +* Minor test cleanups (dpkp #1343) +* Use latest pytest 3.4.0, but drop pytest-sugar due to incompatibility (dpkp #1361) + +Documentation +* Expand metrics docs (jeffwidman #1243) +* Fix docstring (jeffwidman #1261) +* Added controlled thread shutdown to example.py (TheAtomicOption #1268) +* Add license to wheel (jeffwidman #1286) +* Use correct casing for MB (jeffwidman #1298) + +Logging / Error Messages +* Fix two bugs in printing bytes instance (jeffwidman #1296) + + # 1.3.5 (Oct 7, 2017) Bugfixes diff --git a/Makefile b/Makefile index 5f80ccd21..7dfd305e6 100644 --- a/Makefile +++ b/Makefile @@ -23,11 +23,13 @@ test27: build-integration # Test using py.test directly if you want to use local python. Useful for other # platforms that require manual installation for C libraries, ie. Windows. test-local: build-integration - py.test --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF kafka test + KAFKA_VERSION=$(KAFKA_VERSION) SCALA_VERSION=$(SCALA_VERSION) py.test \ + --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF $(FLAGS) kafka test cov-local: build-integration - py.test --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka \ - --cov-config=.covrc --cov-report html kafka test + KAFKA_VERSION=$(KAFKA_VERSION) SCALA_VERSION=$(SCALA_VERSION) py.test \ + --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka \ + --cov-config=.covrc --cov-report html $(FLAGS) kafka test @echo "open file://`pwd`/htmlcov/index.html" # Check the readme for syntax errors, which can lead to invalid formatting on diff --git a/README.rst b/README.rst index d4fc1a9ad..a82573bbf 100644 --- a/README.rst +++ b/README.rst @@ -1,7 +1,7 @@ Kafka Python client ------------------------ -.. image:: https://img.shields.io/badge/kafka-0.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-1.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python @@ -70,6 +70,11 @@ that expose basic message attributes: topic, partition, offset, key, and value: >>> for msg in consumer: ... assert isinstance(msg.value, dict) +>>> # Access record headers. The returned value is a list of tuples +>>> # with str, bytes for key and value +>>> for msg in consumer: +... print (msg.headers) + >>> # Get consumer metrics >>> metrics = consumer.metrics() @@ -112,6 +117,10 @@ for more details. >>> for i in range(1000): ... producer.send('foobar', b'msg %d' % i) +>>> # Include record headers. The format is list of tuples with string key +>>> # and bytes value. +>>> producer.send('foobar', value=b'c29tZSB2YWx1ZQ==', headers=[('content-encoding', b'base64')]) + >>> # Get producer performance metrics >>> metrics = producer.metrics() @@ -141,7 +150,7 @@ 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 KafkaClient.check_version() method that probes a kafka broker and attempts to identify which version it is running -(0.8.0 to 0.11). +(0.8.0 to 1.0). Low-level ********* diff --git a/build_integration.sh b/build_integration.sh index dd875405b..b686fffb5 100755 --- a/build_integration.sh +++ b/build_integration.sh @@ -1,6 +1,6 @@ #!/bin/bash -: ${ALL_RELEASES:="0.8.2.2 0.9.0.1 0.10.1.1 0.10.2.1 0.11.0.2"} +: ${ALL_RELEASES:="0.8.2.2 0.9.0.1 0.10.1.1 0.10.2.1 0.11.0.2 1.0.1"} : ${SCALA_VERSION:=2.11} : ${DIST_BASE_URL:=https://archive.apache.org/dist/kafka/} : ${KAFKA_SRC_GIT:=https://github.com/apache/kafka.git} diff --git a/docs/changelog.rst b/docs/changelog.rst index dc5ca8523..3ed54a34a 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,215 @@ Changelog ========= +1.4.3 (May 26, 2018) +#################### + +Compatibility +------------- +* Fix for python 3.7 support: remove 'async' keyword from SimpleProducer (dpkp #1454) + +Client +------ +* Improve BrokerConnection initialization time (romulorosa #1475) +* Ignore MetadataResponses with empty broker list (dpkp #1506) +* Improve connection handling when bootstrap list is invalid (dpkp #1507) + +Consumer +-------- +* Check for immediate failure when looking up coordinator in heartbeat thread (dpkp #1457) + +Core / Protocol +--------------- +* Always acquire client lock before coordinator lock to avoid deadlocks (dpkp #1464) +* Added AlterConfigs and DescribeConfigs apis (StephenSorriaux #1472) +* Fix CreatePartitionsRequest_v0 (StephenSorriaux #1469) +* Add codec validators to record parser and builder for all formats (tvoinarovskyi #1447) +* Fix MemoryRecord bugs re error handling and add test coverage (tvoinarovskyi #1448) +* Force lz4 to disable Kafka-unsupported block linking when encoding (mnito #1476) +* Stop shadowing `ConnectionError` (jeffwidman #1492) + +Documentation +------------- +* Document methods that return None (jeffwidman #1504) +* Minor doc capitalization cleanup (jeffwidman) +* Adds add_callback/add_errback example to docs (Berkodev #1441) +* Fix KafkaConsumer docstring for request_timeout_ms default (dpkp #1459) + +Test Infrastructure +------------------- +* Skip flakey SimpleProducer test (dpkp) +* Fix skipped integration tests if KAFKA_VERSION unset (dpkp #1453) + +Logging / Error Messages +------------------------ +* Stop using deprecated log.warn() (jeffwidman) +* Change levels for some heartbeat thread logging (dpkp #1456) +* Log Heartbeat thread start / close for debugging (dpkp) + + +1.4.2 (Mar 10, 2018) +#################### + +Bugfixes +-------- +* Close leaked selector in version check (dpkp #1425) +* Fix `BrokerConnection.connection_delay()` to return milliseconds (dpkp #1414) +* Use local copies in `Fetcher._fetchable_partitions` to avoid mutation errors (dpkp #1400) +* Fix error var name in `_unpack` (j2gg0s #1403) +* Fix KafkaConsumer compacted offset handling (dpkp #1397) +* Fix byte size estimation with kafka producer (blakeembrey #1393) +* Fix coordinator timeout in consumer poll interface (braedon #1384) + +Client +------ +* Add `BrokerConnection.connect_blocking()` to improve bootstrap to multi-address hostnames (dpkp #1411) +* Short-circuit `BrokerConnection.close()` if already disconnected (dpkp #1424) +* Only increase reconnect backoff if all addrinfos have been tried (dpkp #1423) +* Make BrokerConnection .host / .port / .afi immutable to avoid incorrect 'metadata changed' checks (dpkp #1422) +* Connect with sockaddrs to support non-zero ipv6 scope ids (dpkp #1433) +* Check timeout type in KafkaClient constructor (asdaraujo #1293) +* Update string representation of SimpleClient (asdaraujo #1293) +* Do not validate `api_version` against known versions (dpkp #1434) + +Consumer +-------- +* Avoid tight poll loop in consumer when brokers are down (dpkp #1415) +* Validate `max_records` in KafkaConsumer.poll (dpkp #1398) +* KAFKA-5512: Awake heartbeat thread when it is time to poll (dpkp #1439) + +Producer +-------- +* Validate that serializers generate bytes-like (or None) data (dpkp #1420) + +Core / Protocol +--------------- +* Support alternative lz4 package: lz4framed (everpcpc #1395) +* Use hardware accelerated CRC32C function if available (tvoinarovskyi #1389) +* Add Admin CreatePartitions API call (alexef #1386) + +Test Infrastructure +------------------- +* Close KafkaConsumer instances during tests (dpkp #1410) +* Introduce new fixtures to prepare for migration to pytest (asdaraujo #1293) +* Removed pytest-catchlog dependency (asdaraujo #1380) +* Fixes racing condition when message is sent to broker before topic logs are created (asdaraujo #1293) +* Add kafka 1.0.1 release to test fixtures (dpkp #1437) + +Logging / Error Messages +------------------------ +* Re-enable logging during broker version check (dpkp #1430) +* Connection logging cleanups (dpkp #1432) +* Remove old CommitFailed error message from coordinator (dpkp #1436) + + +1.4.1 (Feb 9, 2018) +################### + +Bugfixes +-------- +* Fix consumer poll stuck error when no available partition (ckyoog #1375) +* Increase some integration test timeouts (dpkp #1374) +* Use raw in case string overriden (jeffwidman #1373) +* Fix pending completion IndexError bug caused by multiple threads (dpkp #1372) + + +1.4.0 (Feb 6, 2018) +################### + +This is a substantial release. Although there are no known 'showstopper' bugs as of release, +we do recommend you test any planned upgrade to your application prior to running in production. + +Some of the major changes include: + +* We have officially dropped python 2.6 support +* The KafkaConsumer now includes a background thread to handle coordinator heartbeats +* API protocol handling has been separated from networking code into a new class, KafkaProtocol +* Added support for kafka message format v2 +* Refactored DNS lookups during kafka broker connections +* SASL authentication is working (we think) +* Removed several circular references to improve gc on close() + +Thanks to all contributors -- the state of the kafka-python community is strong! + +Detailed changelog are listed below: + +Client +------ +* Fixes for SASL support + + * Refactor SASL/gssapi support (dpkp #1248 #1249 #1257 #1262 #1280) + * Add security layer negotiation to the GSSAPI authentication (asdaraujo #1283) + * Fix overriding sasl_kerberos_service_name in KafkaConsumer / KafkaProducer (natedogs911 #1264) + * Fix typo in _try_authenticate_plain (everpcpc #1333) + * Fix for Python 3 byte string handling in SASL auth (christophelec #1353) + +* Move callback processing from BrokerConnection to KafkaClient (dpkp #1258) +* Use socket timeout of request_timeout_ms to prevent blocking forever on send (dpkp #1281) +* Refactor dns lookup in BrokerConnection (dpkp #1312) +* Read all available socket bytes (dpkp #1332) +* Honor reconnect_backoff in conn.connect() (dpkp #1342) + +Consumer +-------- +* KAFKA-3977: Defer fetch parsing for space efficiency, and to raise exceptions to user (dpkp #1245) +* KAFKA-4034: Avoid unnecessary consumer coordinator lookup (dpkp #1254) +* Handle lookup_coordinator send failures (dpkp #1279) +* KAFKA-3888 Use background thread to process consumer heartbeats (dpkp #1266) +* Improve KafkaConsumer cleanup (dpkp #1339) +* Fix coordinator join_future race condition (dpkp #1338) +* Avoid KeyError when filtering fetchable partitions (dpkp #1344) +* Name heartbeat thread with group_id; use backoff when polling (dpkp #1345) +* KAFKA-3949: Avoid race condition when subscription changes during rebalance (dpkp #1364) +* Fix #1239 regression to avoid consuming duplicate compressed messages from mid-batch (dpkp #1367) + +Producer +-------- +* Fix timestamp not passed to RecordMetadata (tvoinarovskyi #1273) +* Raise non-API exceptions (jeffwidman #1316) +* Fix reconnect_backoff_max_ms default config bug in KafkaProducer (YaoC #1352) + +Core / Protocol +--------------- +* Add kafka.protocol.parser.KafkaProtocol w/ receive and send (dpkp #1230) +* Refactor MessageSet and Message into LegacyRecordBatch to later support v2 message format (tvoinarovskyi #1252) +* Add DefaultRecordBatch implementation aka V2 message format parser/builder. (tvoinarovskyi #1185) +* optimize util.crc32 (ofek #1304) +* Raise better struct pack/unpack errors (jeffwidman #1320) +* Add Request/Response structs for kafka broker 1.0.0 (dpkp #1368) + +Bugfixes +-------- +* use python standard max value (lukekingbru #1303) +* changed for to use enumerate() (TheAtomicOption #1301) +* Explicitly check for None rather than falsey (jeffwidman #1269) +* Minor Exception cleanup (jeffwidman #1317) +* Use non-deprecated exception handling (jeffwidman a699f6a) +* Remove assertion with side effect in client.wakeup() (bgedik #1348) +* use absolute imports everywhere (kevinkjt2000 #1362) + +Test Infrastructure +------------------- +* Use 0.11.0.2 kafka broker for integration testing (dpkp #1357 #1244) +* Add a Makefile to help build the project, generate docs, and run tests (tvoinarovskyi #1247) +* Add fixture support for 1.0.0 broker (dpkp #1275) +* Add kafka 1.0.0 to travis integration tests (dpkp #1365) +* Change fixture default host to localhost (asdaraujo #1305) +* Minor test cleanups (dpkp #1343) +* Use latest pytest 3.4.0, but drop pytest-sugar due to incompatibility (dpkp #1361) + +Documentation +------------- +* Expand metrics docs (jeffwidman #1243) +* Fix docstring (jeffwidman #1261) +* Added controlled thread shutdown to example.py (TheAtomicOption #1268) +* Add license to wheel (jeffwidman #1286) +* Use correct casing for MB (jeffwidman #1298) + +Logging / Error Messages +------------------------ +* Fix two bugs in printing bytes instance (jeffwidman #1296) + + 1.3.5 (Oct 7, 2017) #################### diff --git a/docs/compatibility.rst b/docs/compatibility.rst index a832ae631..1771d8f76 100644 --- a/docs/compatibility.rst +++ b/docs/compatibility.rst @@ -1,12 +1,12 @@ Compatibility ------------- -.. image:: https://img.shields.io/badge/kafka-0.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-1.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/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.11 +kafka-python is compatible with (and tested against) broker versions 1.0 through 0.8.0 . kafka-python is not compatible with the 0.8.2-beta release. kafka-python is tested on python 2.7, 3.4, 3.5, 3.6 and pypy. diff --git a/docs/index.rst b/docs/index.rst index f84992a77..51c39f59d 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,7 +1,7 @@ kafka-python ############ -.. image:: https://img.shields.io/badge/kafka-0.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-1.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python @@ -136,7 +136,7 @@ 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.11). +attempts to identify which version it is running (0.8.0 to 1.0). Low-level diff --git a/docs/install.rst b/docs/install.rst index cc0e82d68..fe740f660 100644 --- a/docs/install.rst +++ b/docs/install.rst @@ -70,3 +70,16 @@ Install the `python-snappy` module .. code:: bash pip install python-snappy + + +Optional crc32c install +*********************** +Highly recommended if you are using Kafka 11+ brokers. For those `kafka-python` +uses a new message protocol version, that requires calculation of `crc32c`, +which differs from `zlib.crc32` hash implementation. By default `kafka-python` +calculates it in pure python, which is quite slow. To speed it up we optionally +support https://pypi.python.org/pypi/crc32c package if it's installed. + +.. code:: bash + + pip install crc32c diff --git a/docs/simple.rst b/docs/simple.rst index 8192a8b76..afdb9756c 100644 --- a/docs/simple.rst +++ b/docs/simple.rst @@ -49,7 +49,7 @@ Asynchronous Mode # To send messages asynchronously client = SimpleClient('localhost:9092') - producer = SimpleProducer(client, async=True) + producer = SimpleProducer(client, async_send=True) producer.send_messages('my-topic', b'async message') # To send messages in batch. You can use any of the available @@ -60,7 +60,7 @@ Asynchronous Mode # * If the producer dies before the messages are sent, there will be losses # * Call producer.stop() to send the messages and cleanup producer = SimpleProducer(client, - async=True, + async_send=True, batch_send_every_n=20, batch_send_every_t=60) @@ -73,7 +73,7 @@ Synchronous Mode # To send messages synchronously client = SimpleClient('localhost:9092') - producer = SimpleProducer(client, async=False) + producer = SimpleProducer(client, async_send=False) # Note that the application is responsible for encoding messages to type bytes producer.send_messages('my-topic', b'some message') @@ -88,7 +88,7 @@ Synchronous Mode # 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, + async_send=False, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE, ack_timeout=2000, sync_fail_on_error=False) diff --git a/docs/usage.rst b/docs/usage.rst index 22fe20d5c..1cf1aa414 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -91,6 +91,18 @@ KafkaProducer for _ in range(100): producer.send('my-topic', b'msg') + def on_send_success(record_metadata): + print(record_metadata.topic) + print(record_metadata.partition) + print(record_metadata.offset) + + def on_send_error(excp): + log.error('I am an errback', exc_info=excp) + # handle exception + + # produce asynchronously with callbacks + producer.send('my-topic', b'raw_bytes').add_callback(on_send_success).add_errback(on_send_error) + # block until all async messages are sent producer.flush() diff --git a/kafka/__init__.py b/kafka/__init__.py index 6a8041825..897ebb095 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -1,7 +1,7 @@ from __future__ import absolute_import __title__ = 'kafka' -from .version import __version__ +from kafka.version import __version__ __author__ = 'Dana Powers' __license__ = 'Apache License 2.0' __copyright__ = 'Copyright 2016 Dana Powers, David Arthur, and Contributors' @@ -25,8 +25,8 @@ def emit(self, record): from kafka.protocol import ( create_message, create_gzip_message, create_snappy_message) from kafka.partitioner import RoundRobinPartitioner, HashedPartitioner, Murmur2Partitioner -from kafka.structs import TopicPartition, OffsetAndMetadata from kafka.serializer import Serializer, Deserializer +from kafka.structs import TopicPartition, OffsetAndMetadata # To be deprecated when KafkaProducer interface is released from kafka.client import SimpleClient @@ -50,5 +50,5 @@ def __init__(self, *args, **kwargs): 'SimpleClient', 'SimpleProducer', 'KeyedProducer', 'RoundRobinPartitioner', 'HashedPartitioner', 'create_message', 'create_gzip_message', 'create_snappy_message', - 'SimpleConsumer', 'MultiProcessConsumer', + 'SimpleConsumer', 'MultiProcessConsumer', 'ConsumerRebalanceListener', ] diff --git a/kafka/client.py b/kafka/client.py index 22918ac51..789d4da3d 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -11,7 +11,7 @@ from kafka.vendor import six import kafka.errors -from kafka.errors import (UnknownError, ConnectionError, FailedPayloadsError, +from kafka.errors import (UnknownError, KafkaConnectionError, FailedPayloadsError, KafkaTimeoutError, KafkaUnavailableError, LeaderNotAvailableError, UnknownTopicOrPartitionError, NotLeaderForPartitionError, ReplicaNotAvailableError) @@ -71,19 +71,9 @@ def _get_conn(self, host, port, afi): ) conn = self._conns[host_key] - conn.connect() - if conn.connected(): - return conn - - timeout = time.time() + self.timeout - while time.time() < timeout and conn.connecting(): - if conn.connect() is ConnectionStates.CONNECTED: - break - else: - time.sleep(0.05) - else: + if not conn.connect_blocking(self.timeout): conn.close() - raise ConnectionError("%s:%s (%s)" % (host, port, afi)) + raise KafkaConnectionError("%s:%s (%s)" % (host, port, afi)) return conn def _get_leader_for_partition(self, topic, partition): @@ -166,7 +156,7 @@ def _send_broker_unaware_request(self, payloads, encoder_fn, decoder_fn): for (host, port, afi) in hosts: try: conn = self._get_conn(host, port, afi) - except ConnectionError: + except KafkaConnectionError: log.warning("Skipping unconnected connection: %s:%s (AFI %s)", host, port, afi) continue @@ -252,7 +242,7 @@ def failed_payloads(payloads): host, port, afi = get_ip_port_afi(broker.host) try: conn = self._get_conn(host, broker.port, afi) - except ConnectionError: + except KafkaConnectionError: refresh_metadata = True failed_payloads(broker_payloads) continue @@ -354,8 +344,8 @@ def _send_consumer_aware_request(self, group, payloads, encoder_fn, decoder_fn): try: host, port, afi = get_ip_port_afi(broker.host) conn = self._get_conn(host, broker.port, afi) - except ConnectionError as e: - log.warning('ConnectionError attempting to send request %s ' + except KafkaConnectionError as e: + log.warning('KafkaConnectionError attempting to send request %s ' 'to server %s: %s', request_id, broker, e) for payload in payloads: @@ -404,7 +394,7 @@ def _send_consumer_aware_request(self, group, payloads, encoder_fn, decoder_fn): return [responses[tp] for tp in original_ordering] def __repr__(self): - return '' % (self.client_id) + return '' % (self.client_id) def _raise_on_response_error(self, resp): diff --git a/kafka/client_async.py b/kafka/client_async.py index 2e78e3b3a..5a161bb6a 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -13,26 +13,26 @@ import selectors # pylint: disable=import-error except ImportError: # vendored backport module - from .vendor import selectors34 as selectors + from kafka.vendor import selectors34 as selectors import socket import time from kafka.vendor import six -from .cluster import ClusterMetadata -from .conn import BrokerConnection, ConnectionStates, collect_hosts, get_ip_port_afi -from . import errors as Errors -from .future import Future -from .metrics import AnonMeasurable -from .metrics.stats import Avg, Count, Rate -from .metrics.stats.rate import TimeUnit -from .protocol.metadata import MetadataRequest -from .util import Dict, WeakMethod +from kafka.cluster import ClusterMetadata +from kafka.conn import BrokerConnection, ConnectionStates, collect_hosts, get_ip_port_afi +from kafka import errors as Errors +from kafka.future import Future +from kafka.metrics import AnonMeasurable +from kafka.metrics.stats import Avg, Count, Rate +from kafka.metrics.stats.rate import TimeUnit +from kafka.protocol.metadata import MetadataRequest +from kafka.util import Dict, WeakMethod # Although this looks unused, it actually monkey-patches socket.socketpair() # and should be left in as long as we're using socket.socketpair() in this file -from .vendor import socketpair -from .version import __version__ +from kafka.vendor import socketpair +from kafka.version import __version__ if six.PY2: ConnectionError = None @@ -78,7 +78,12 @@ class KafkaClient(object): resulting in a random range between 20% below and 20% above the computed value. Default: 1000. request_timeout_ms (int): Client request timeout in milliseconds. - Default: 40000. + Default: 30000. + connections_max_idle_ms: Close idle connections after the number of + milliseconds specified by this config. The broker closes idle + connections after connections.max.idle.ms, so this avoids hitting + unexpected socket disconnected errors on the client. + Default: 540000 retry_backoff_ms (int): Milliseconds to backoff when retrying on errors. Default: 100. max_in_flight_requests_per_connection (int): Requests are pipelined @@ -99,30 +104,29 @@ class KafkaClient(object): brokers or partitions. Default: 300000 security_protocol (str): Protocol used to communicate with brokers. Valid values are: PLAINTEXT, SSL. Default: PLAINTEXT. - ssl_context (ssl.SSLContext): pre-configured SSLContext for wrapping + ssl_context (ssl.SSLContext): Pre-configured SSLContext for wrapping socket connections. If provided, all other ssl_* configurations will be ignored. Default: None. - ssl_check_hostname (bool): flag to configure whether ssl handshake - should verify that the certificate matches the brokers hostname. - default: true. - ssl_cafile (str): optional filename of ca file to use in certificate - veriication. default: none. - ssl_certfile (str): optional filename of file in pem format containing - the client certificate, as well as any ca certificates needed to - establish the certificate's authenticity. default: none. - ssl_keyfile (str): optional filename containing the client private key. - default: none. - ssl_password (str): optional password to be used when loading the - certificate chain. default: none. - ssl_crlfile (str): optional filename containing the CRL to check for + ssl_check_hostname (bool): Flag to configure whether SSL handshake + should verify that the certificate matches the broker's hostname. + Default: True. + ssl_cafile (str): Optional filename of CA file to use in certificate + veriication. Default: None. + ssl_certfile (str): Optional filename of file in PEM format containing + the client certificate, as well as any CA certificates needed to + establish the certificate's authenticity. Default: None. + ssl_keyfile (str): Optional filename containing the client private key. + Default: None. + ssl_password (str): Optional password to be used when loading the + certificate chain. Default: None. + ssl_crlfile (str): Optional filename containing the CRL to check for certificate expiration. By default, no CRL check is done. When providing a file, only the leaf certificate will be checked against this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+. - default: none. + Default: None. api_version (tuple): Specify which Kafka API version to use. If set to None, KafkaClient will attempt to infer the broker version by - probing various APIs. For the full list of supported versions, - see KafkaClient.API_VERSIONS. Default: None + probing various APIs. Example: (0, 10, 2). Default: None api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker api version. Only applies if api_version is None @@ -141,12 +145,15 @@ class KafkaClient(object): Default: None sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' + sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI + sasl mechanism handshake. Default: one of bootstrap servers """ DEFAULT_CONFIG = { 'bootstrap_servers': 'localhost', + 'bootstrap_topics_filter': set(), 'client_id': 'kafka-python-' + __version__, - 'request_timeout_ms': 40000, + 'request_timeout_ms': 30000, 'connections_max_idle_ms': 9 * 60 * 1000, 'reconnect_backoff_ms': 50, 'reconnect_backoff_max_ms': 1000, @@ -175,16 +182,8 @@ class KafkaClient(object): 'sasl_plain_username': None, 'sasl_plain_password': None, 'sasl_kerberos_service_name': 'kafka', + 'sasl_kerberos_domain_name': None } - API_VERSIONS = [ - (0, 10, 1), - (0, 10, 0), - (0, 10), - (0, 9), - (0, 8, 2), - (0, 8, 1), - (0, 8, 0) - ] def __init__(self, **configs): self.config = copy.copy(self.DEFAULT_CONFIG) @@ -192,11 +191,6 @@ def __init__(self, **configs): if key in configs: self.config[key] = configs[key] - if self.config['api_version'] is not None: - assert self.config['api_version'] in self.API_VERSIONS, ( - 'api_version [{0}] must be one of: {1}'.format( - self.config['api_version'], str(self.API_VERSIONS))) - self.cluster = ClusterMetadata(**self.config) self._topics = set() # empty set will fetch all topic metadata self._metadata_refresh_in_progress = False @@ -246,9 +240,15 @@ def _bootstrap(self, hosts): self._last_bootstrap = time.time() if self.config['api_version'] is None or self.config['api_version'] < (0, 10): - metadata_request = MetadataRequest[0]([]) + if self.config['bootstrap_topics_filter']: + metadata_request = MetadataRequest[0](list(self.config['bootstrap_topics_filter'])) + else: + metadata_request = MetadataRequest[0]([]) else: - metadata_request = MetadataRequest[1](None) + if self.config['bootstrap_topics_filter']: + metadata_request = MetadataRequest[1](list(self.config['bootstrap_topics_filter'])) + else: + metadata_request = MetadataRequest[1](None) for host, port, afi in hosts: log.debug("Attempting to bootstrap via node at %s:%s", host, port) @@ -257,11 +257,7 @@ def _bootstrap(self, hosts): state_change_callback=cb, node_id='bootstrap', **self.config) - bootstrap.connect() - while bootstrap.connecting(): - self._selector.select(1) - bootstrap.connect() - if not bootstrap.connected(): + if not bootstrap.connect_blocking(): bootstrap.close() continue future = bootstrap.send(metadata_request) @@ -545,6 +541,8 @@ def poll(self, timeout_ms=None, future=None): timeout_ms = 100 elif timeout_ms is None: timeout_ms = self.config['request_timeout_ms'] + elif not isinstance(timeout_ms, (int, float)): + raise RuntimeError('Invalid type for timeout: %s' % type(timeout_ms)) # Loop for futures, break after first loop if None responses = [] @@ -619,7 +617,7 @@ def _poll(self, timeout): log.warning('Protocol out of sync on %r, closing', conn) except socket.error: pass - conn.close(Errors.ConnectionError('Socket EVENT_READ without in-flight-requests')) + conn.close(Errors.KafkaConnectionError('Socket EVENT_READ without in-flight-requests')) continue self._idle_expiry_manager.update(conn.node_id) @@ -665,8 +663,14 @@ def in_flight_request_count(self, node_id=None): def _fire_pending_completed_requests(self): responses = [] - while self._pending_completion: - response, future = self._pending_completion.popleft() + while True: + try: + # We rely on deque.popleft remaining threadsafe + # to allow both the heartbeat thread and the main thread + # to process responses + response, future = self._pending_completion.popleft() + except IndexError: + break future.success(response) responses.append(response) return responses @@ -836,7 +840,7 @@ def check_version(self, node_id=None, timeout=2, strict=False): self._refresh_on_disconnects = False try: remaining = end - time.time() - version = conn.check_version(timeout=remaining, strict=strict) + version = conn.check_version(timeout=remaining, strict=strict, topics=list(self.config['bootstrap_topics_filter'])) return version except Errors.NodeNotReadyError: # Only raise to user if this is a node-specific request diff --git a/kafka/cluster.py b/kafka/cluster.py index d646fdfee..8078eb7cf 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -8,9 +8,9 @@ from kafka.vendor import six -from . import errors as Errors -from .future import Future -from .structs import BrokerMetadata, PartitionMetadata, TopicPartition +from kafka import errors as Errors +from kafka.future import Future +from kafka.structs import BrokerMetadata, PartitionMetadata, TopicPartition log = logging.getLogger(__name__) @@ -96,6 +96,7 @@ def available_partitions_for_topic(self, topic): Returns: set: {partition (int), ...} + None if topic not found. """ if topic not in self._partitions: return None @@ -119,6 +120,7 @@ def partitions_for_broker(self, broker_id): Returns: set: {TopicPartition, ...} + None if the broker either has no partitions or does not exist. """ return self._broker_partitions.get(broker_id) @@ -130,6 +132,7 @@ def coordinator_for_group(self, group): Returns: int: node_id for group coordinator + None if the group does not exist. """ return self._groups.get(group) @@ -211,7 +214,8 @@ def update_metadata(self, metadata): return self.failed_update(error) if not metadata.brokers: - log.warning("No broker metadata found in MetadataResponse") + log.warning("No broker metadata found in MetadataResponse -- ignoring.") + return self.failed_update(Errors.MetadataEmptyBrokerList(metadata)) _new_brokers = {} for broker in metadata.brokers: @@ -291,6 +295,13 @@ def update_metadata(self, metadata): for listener in self._listeners: listener(self) + if self.need_all_topic_metadata: + # the listener may change the interested topics, + # which could cause another metadata refresh. + # If we have already fetched all topics, however, + # another fetch should be unnecessary. + self._need_update = False + def add_listener(self, listener): """Add a callback function to be called on each metadata update""" self._listeners.add(listener) diff --git a/kafka/codec.py b/kafka/codec.py index a527b4273..4d180ddd3 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -18,6 +18,18 @@ try: import lz4.frame as lz4 + + def _lz4_compress(payload, **kwargs): + # Kafka does not support LZ4 dependent blocks + try: + # For lz4>=0.12.0 + kwargs.pop('block_linked', None) + return lz4.compress(payload, block_linked=False, **kwargs) + except TypeError: + # For earlier versions of lz4 + kwargs.pop('block_mode', None) + return lz4.compress(payload, block_mode=1, **kwargs) + except ImportError: lz4 = None @@ -26,6 +38,11 @@ except ImportError: lz4f = None +try: + import lz4framed +except ImportError: + lz4framed = None + try: import xxhash except ImportError: @@ -46,6 +63,8 @@ def has_lz4(): return True if lz4f is not None: return True + if lz4framed is not None: + return True return False @@ -195,9 +214,11 @@ def snappy_decode(payload): if lz4: - lz4_encode = lz4.compress # pylint: disable-msg=no-member + lz4_encode = _lz4_compress # pylint: disable-msg=no-member elif lz4f: lz4_encode = lz4f.compressFrame # pylint: disable-msg=no-member +elif lz4framed: + lz4_encode = lz4framed.compress # pylint: disable-msg=no-member else: lz4_encode = None @@ -220,6 +241,8 @@ def lz4f_decode(payload): lz4_decode = lz4.decompress # pylint: disable-msg=no-member elif lz4f: lz4_decode = lz4f_decode +elif lz4framed: + lz4_decode = lz4framed.decompress # pylint: disable-msg=no-member else: lz4_decode = None diff --git a/kafka/conn.py b/kafka/conn.py index 5ff27d5ce..ccaa2ed62 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -12,7 +12,7 @@ import selectors # pylint: disable=import-error except ImportError: # vendored backport module - from .vendor import selectors34 as selectors + from kafka.vendor import selectors34 as selectors import socket import struct @@ -78,6 +78,14 @@ class SSLWantWriteError(Exception): gssapi = None GSSError = None + +AFI_NAMES = { + socket.AF_UNSPEC: "unspecified", + socket.AF_INET: "IPv4", + socket.AF_INET6: "IPv6", +} + + class ConnectionStates(object): DISCONNECTING = '' DISCONNECTED = '' @@ -108,7 +116,7 @@ class BrokerConnection(object): resulting in a random range between 20% below and 20% above the computed value. Default: 1000. request_timeout_ms (int): Client request timeout in milliseconds. - Default: 40000. + Default: 30000. 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. @@ -168,12 +176,14 @@ class BrokerConnection(object): Default: None sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' + sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI + sasl mechanism handshake. Default: one of bootstrap servers """ DEFAULT_CONFIG = { 'client_id': 'kafka-python-' + __version__, 'node_id': 0, - 'request_timeout_ms': 40000, + 'request_timeout_ms': 30000, 'reconnect_backoff_ms': 50, 'reconnect_backoff_max_ms': 1000, 'max_in_flight_requests_per_connection': 5, @@ -198,19 +208,18 @@ class BrokerConnection(object): 'sasl_mechanism': 'PLAIN', 'sasl_plain_username': None, 'sasl_plain_password': None, - 'sasl_kerberos_service_name': 'kafka' + 'sasl_kerberos_service_name': 'kafka', + 'sasl_kerberos_domain_name': None } SECURITY_PROTOCOLS = ('PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL') SASL_MECHANISMS = ('PLAIN', 'GSSAPI') def __init__(self, host, port, afi, **configs): - self.hostname = host self.host = host self.port = port self.afi = afi - self._init_host = host - self._init_port = port - self._init_afi = afi + self._sock_afi = afi + self._sock_addr = None self.in_flight_requests = collections.deque() self._api_versions = None @@ -257,37 +266,72 @@ def __init__(self, host, port, afi, **configs): self._ssl_context = self.config['ssl_context'] self._sasl_auth_future = None self.last_attempt = 0 - self._gai = None + self._gai = [] self._sensors = None if self.config['metrics']: self._sensors = BrokerConnectionMetrics(self.config['metrics'], self.config['metric_group_prefix'], self.node_id) - def _next_afi_host_port(self): + def _dns_lookup(self): + self._gai = dns_lookup(self.host, self.port, self.afi) if not self._gai: - self._gai = dns_lookup(self._init_host, self._init_port, self._init_afi) - if not self._gai: - log.error('DNS lookup failed for %s:%i (%s)', - self._init_host, self._init_port, self._init_afi) - return + log.error('DNS lookup failed for %s:%i (%s)', + self.host, self.port, self.afi) + return False + return True + def _next_afi_sockaddr(self): + if not self._gai: + if not self._dns_lookup(): + return afi, _, __, ___, sockaddr = self._gai.pop(0) - host, port = sockaddr[:2] - return (afi, host, port) + return (afi, sockaddr) + + def connect_blocking(self, timeout=float('inf')): + if self.connected(): + return True + timeout += time.time() + # First attempt to perform dns lookup + # note that the underlying interface, socket.getaddrinfo, + # has no explicit timeout so we may exceed the user-specified timeout + self._dns_lookup() + + # Loop once over all returned dns entries + selector = None + while self._gai: + while time.time() < timeout: + self.connect() + if self.connected(): + if selector is not None: + selector.close() + return True + elif self.connecting(): + if selector is None: + selector = self.config['selector']() + selector.register(self._sock, selectors.EVENT_WRITE) + selector.select(1) + elif self.disconnected(): + if selector is not None: + selector.close() + selector = None + break + else: + break + return False def connect(self): """Attempt to connect and return ConnectionState""" if self.state is ConnectionStates.DISCONNECTED and not self.blacked_out(): self.last_attempt = time.time() - next_lookup = self._next_afi_host_port() + next_lookup = self._next_afi_sockaddr() if not next_lookup: - self.close(Errors.ConnectionError('DNS failure')) + self.close(Errors.KafkaConnectionError('DNS failure')) return else: log.debug('%s: creating new socket', self) - self.afi, self.host, self.port = next_lookup - self._sock = socket.socket(self.afi, socket.SOCK_STREAM) + self._sock_afi, self._sock_addr = next_lookup + self._sock = socket.socket(self._sock_afi, socket.SOCK_STREAM) for option in self.config['socket_options']: log.debug('%s: setting socket option %s', self, option) @@ -301,7 +345,8 @@ def connect(self): # so we need to double check that we are still connecting before if self.connecting(): self.config['state_change_callback'](self) - log.info('%s: connecting to %s:%d', self, self.host, self.port) + log.info('%s: connecting to %s:%d [%s %s]', self, self.host, + self.port, self._sock_addr, AFI_NAMES[self._sock_afi]) if self.state is ConnectionStates.CONNECTING: # in non-blocking mode, use repeated calls to socket.connect_ex @@ -309,7 +354,7 @@ def connect(self): request_timeout = self.config['request_timeout_ms'] / 1000.0 ret = None try: - ret = self._sock.connect_ex((self.host, self.port)) + ret = self._sock.connect_ex(self._sock_addr) except socket.error as err: ret = err.errno @@ -324,7 +369,7 @@ def connect(self): self.state = ConnectionStates.AUTHENTICATING else: # security_protocol PLAINTEXT - log.debug('%s: Connection complete.', self) + log.info('%s: Connection complete.', self) self.state = ConnectionStates.CONNECTED self._reset_reconnect_backoff() self.config['state_change_callback'](self) @@ -334,12 +379,13 @@ def connect(self): elif ret not in (errno.EINPROGRESS, errno.EALREADY, errno.EWOULDBLOCK, 10022): log.error('Connect attempt to %s returned error %s.' ' Disconnecting.', self, ret) - self.close(Errors.ConnectionError(ret)) + errstr = errno.errorcode.get(ret, 'UNKNOWN') + self.close(Errors.KafkaConnectionError('{} {}'.format(ret, errstr))) # Connection timed out elif time.time() > request_timeout + self.last_attempt: log.error('Connection attempt to %s timed out', self) - self.close(Errors.ConnectionError('timeout')) + self.close(Errors.KafkaConnectionError('timeout')) # Needs retry else: @@ -352,7 +398,7 @@ def connect(self): log.debug('%s: initiating SASL authentication', self) self.state = ConnectionStates.AUTHENTICATING else: - log.debug('%s: Connection complete.', self) + log.info('%s: Connection complete.', self) self.state = ConnectionStates.CONNECTED self.config['state_change_callback'](self) @@ -361,7 +407,7 @@ def connect(self): if self._try_authenticate(): # _try_authenticate has side-effects: possibly disconnected on socket errors if self.state is ConnectionStates.AUTHENTICATING: - log.debug('%s: Connection complete.', self) + log.info('%s: Connection complete.', self) self.state = ConnectionStates.CONNECTED self._reset_reconnect_backoff() self.config['state_change_callback'](self) @@ -400,7 +446,7 @@ def _wrap_ssl(self): try: self._sock = self._ssl_context.wrap_socket( self._sock, - server_hostname=self.hostname, + server_hostname=self.host, do_handshake_on_connect=False) except ssl.SSLError as e: log.exception('%s: Failed to wrap socket in SSLContext!', self) @@ -416,7 +462,7 @@ def _try_handshake(self): pass except (SSLZeroReturnError, ConnectionError, SSLEOFError): log.warning('SSL connection closed by server during handshake.') - self.close(Errors.ConnectionError('SSL connection closed by server during handshake')) + self.close(Errors.KafkaConnectionError('SSL connection closed by server during handshake')) # Other SSLErrors will be raised to user return False @@ -441,7 +487,7 @@ def _try_authenticate(self): return False elif self._sasl_auth_future.failed(): ex = self._sasl_auth_future.exception - if not isinstance(ex, Errors.ConnectionError): + if not isinstance(ex, Errors.KafkaConnectionError): raise ex # pylint: disable-msg=raising-bad-type return self._sasl_auth_future.succeeded() @@ -511,8 +557,8 @@ def _try_authenticate_plain(self, future): data = self._recv_bytes_blocking(4) except ConnectionError as e: - log.exception("%s: Error receiving reply from server", self) - error = Errors.ConnectionError("%s: %s" % (self, e)) + log.exception("%s: Error receiving reply from server", self) + error = Errors.KafkaConnectionError("%s: %s" % (self, e)) self.close(error=error) return future.failure(error) @@ -524,7 +570,8 @@ def _try_authenticate_plain(self, future): return future.success(True) def _try_authenticate_gssapi(self, future): - auth_id = self.config['sasl_kerberos_service_name'] + '@' + self.hostname + kerberos_damin_name = self.config['sasl_kerberos_domain_name'] or self.host + auth_id = self.config['sasl_kerberos_service_name'] + '@' + kerberos_damin_name gssapi_name = gssapi.Name( auth_id, name_type=gssapi.NameType.hostbased_service @@ -574,7 +621,7 @@ def _try_authenticate_gssapi(self, future): except ConnectionError as e: log.exception("%s: Error receiving reply from server", self) - error = Errors.ConnectionError("%s: %s" % (self, e)) + error = Errors.KafkaConnectionError("%s: %s" % (self, e)) self.close(error=error) return future.failure(error) except Exception as e: @@ -594,9 +641,16 @@ def blacked_out(self): return False def connection_delay(self): - time_waited_ms = time.time() - (self.last_attempt or 0) + """ + Return the number of milliseconds to wait, based on the connection + state, before attempting to send data. When disconnected, this respects + the reconnect backoff time. When connecting, returns 0 to allow + non-blocking connect to finish. When connected, returns a very large + number to handle slow/stalled connections. + """ + time_waited = time.time() - (self.last_attempt or 0) if self.state is ConnectionStates.DISCONNECTED: - return max(self._reconnect_backoff - time_waited_ms, 0) + return max(self._reconnect_backoff - time_waited, 0) * 1000 elif self.connecting(): return 0 else: @@ -622,6 +676,9 @@ def _reset_reconnect_backoff(self): self._reconnect_backoff = self.config['reconnect_backoff_ms'] / 1000.0 def _update_reconnect_backoff(self): + # Do not mark as failure if there are more dns entries available to try + if len(self._gai) > 0: + return if self.config['reconnect_backoff_max_ms'] > self.config['reconnect_backoff_ms']: self._failures += 1 self._reconnect_backoff = self.config['reconnect_backoff_ms'] * 2 ** (self._failures - 1) @@ -644,12 +701,15 @@ def close(self, error=None): Arguments: error (Exception, optional): pending in-flight-requests will be failed with this exception. - Default: kafka.errors.ConnectionError. + Default: kafka.errors.KafkaConnectionError. """ + if self.state is ConnectionStates.DISCONNECTED: + if error is not None: + log.warning('%s: Duplicate close() with error: %s', self, error) + return log.info('%s: Closing connection. %s', self, error or '') - if self.state is not ConnectionStates.DISCONNECTED: - self.state = ConnectionStates.DISCONNECTING - self.config['state_change_callback'](self) + self.state = ConnectionStates.DISCONNECTING + self.config['state_change_callback'](self) self._update_reconnect_backoff() self._close_socket() self.state = ConnectionStates.DISCONNECTED @@ -673,7 +733,7 @@ def send(self, request): if self.connecting(): return future.failure(Errors.NodeNotReadyError(str(self))) elif not self.connected(): - return future.failure(Errors.ConnectionError(str(self))) + return future.failure(Errors.KafkaConnectionError(str(self))) elif not self.can_send_more(): return future.failure(Errors.TooManyInFlightRequests(str(self))) return self._send(request) @@ -693,7 +753,7 @@ def _send(self, request): self._sensors.bytes_sent.record(total_bytes) except ConnectionError as e: log.exception("Error sending %s to %s", request, self) - error = Errors.ConnectionError("%s: %s" % (self, e)) + error = Errors.KafkaConnectionError("%s: %s" % (self, e)) self.close(error=error) return future.failure(error) log.debug('%s Request %d: %s', self, correlation_id, request) @@ -721,7 +781,7 @@ def recv(self): # If requests are pending, we should close the socket and # fail all the pending request futures if self.in_flight_requests: - self.close(Errors.ConnectionError('Socket not connected during recv with in-flight-requests')) + self.close(Errors.KafkaConnectionError('Socket not connected during recv with in-flight-requests')) return () elif not self.in_flight_requests: @@ -761,7 +821,7 @@ def _recv(self): # without an exception raised if not data: log.error('%s: socket disconnected', self) - self.close(error=Errors.ConnectionError('socket disconnected')) + self.close(error=Errors.KafkaConnectionError('socket disconnected')) return [] else: recvd.append(data) @@ -773,7 +833,7 @@ def _recv(self): break log.exception('%s: Error receiving network data' ' closing socket', self) - self.close(error=Errors.ConnectionError(e)) + self.close(error=Errors.KafkaConnectionError(e)) return [] except BlockingIOError: if six.PY3: @@ -818,6 +878,7 @@ def _infer_broker_version_from_api_versions(self, api_versions): # in reverse order. As soon as we find one that works, return it test_cases = [ # format (, ) + ((1, 0, 0), MetadataRequest[5]), ((0, 11, 0), MetadataRequest[4]), ((0, 10, 2), OffsetFetchRequest[2]), ((0, 10, 1), MetadataRequest[2]), @@ -835,13 +896,15 @@ def _infer_broker_version_from_api_versions(self, api_versions): # so if all else fails, choose that return (0, 10, 0) - def check_version(self, timeout=2, strict=False): + def check_version(self, timeout=2, strict=False, topics=[]): """Attempt to guess the broker version. Note: This is a blocking call. Returns: version tuple, i.e. (0, 10), (0, 9), (0, 8, 2), ... """ + timeout_at = time.time() + timeout + log.info('Probing node %s broker version', self.node_id) # Monkeypatch some connection configurations to avoid timeouts override_config = { 'request_timeout_ms': timeout * 1000, @@ -857,19 +920,8 @@ def check_version(self, timeout=2, strict=False): # vanilla MetadataRequest. If the server did not recognize the first # request, both will be failed with a ConnectionError that wraps # socket.error (32, 54, or 104) - from .protocol.admin import ApiVersionRequest, ListGroupsRequest - from .protocol.commit import OffsetFetchRequest, GroupCoordinatorRequest - - # Socket errors are logged as exceptions and can alarm users. Mute them - from logging import Filter - - class ConnFilter(Filter): - def filter(self, record): - if record.funcName == 'check_version': - return True - return False - log_filter = ConnFilter() - log.addFilter(log_filter) + from kafka.protocol.admin import ApiVersionRequest, ListGroupsRequest + from kafka.protocol.commit import OffsetFetchRequest, GroupCoordinatorRequest test_cases = [ # All cases starting from 0.10 will be based on ApiVersionResponse @@ -877,22 +929,12 @@ def filter(self, record): ((0, 9), ListGroupsRequest[0]()), ((0, 8, 2), GroupCoordinatorRequest[0]('kafka-python-default-group')), ((0, 8, 1), OffsetFetchRequest[0]('kafka-python-default-group', [])), - ((0, 8, 0), MetadataRequest[0]([])), + ((0, 8, 0), MetadataRequest[0](topics)), ] - def connect(): - self.connect() - if self.connected(): - return - timeout_at = time.time() + timeout - while time.time() < timeout_at and self.connecting(): - if self.connect() is ConnectionStates.CONNECTED: - return - time.sleep(0.05) - raise Errors.NodeNotReadyError() - for version, request in test_cases: - connect() + if not self.connect_blocking(timeout_at - time.time()): + raise Errors.NodeNotReadyError() f = self.send(request) # HACK: sleeping to wait for socket to send bytes time.sleep(0.1) @@ -903,14 +945,15 @@ def connect(): # the attempt to write to a disconnected socket should # immediately fail and allow us to infer that the prior # request was unrecognized - mr = self.send(MetadataRequest[0]([])) + mr = self.send(MetadataRequest[0](topics)) selector = self.config['selector']() selector.register(self._sock, selectors.EVENT_READ) while not (f.is_done and mr.is_done): + selector.select(1) for response, future in self.recv(): future.success(response) - selector.select(1) + selector.close() if f.succeeded(): if isinstance(request, ApiVersionRequest[0]): @@ -949,14 +992,14 @@ def connect(): else: raise Errors.UnrecognizedBrokerVersion() - log.removeFilter(log_filter) for key in stashed: self.config[key] = stashed[key] return version - def __repr__(self): - return "" % ( - self.node_id, self.hostname, self.host, self.port) + def __str__(self): + return "" % ( + self.node_id, self.host, self.port, self.state, + AFI_NAMES[self._sock_afi], self._sock_addr) class BrokerConnectionMetrics(object): diff --git a/kafka/consumer/__init__.py b/kafka/consumer/__init__.py index 36c8ff094..4b900ac8c 100644 --- a/kafka/consumer/__init__.py +++ b/kafka/consumer/__init__.py @@ -1,8 +1,8 @@ from __future__ import absolute_import -from .simple import SimpleConsumer -from .multiprocess import MultiProcessConsumer -from .group import KafkaConsumer +from kafka.consumer.simple import SimpleConsumer +from kafka.consumer.multiprocess import MultiProcessConsumer +from kafka.consumer.group import KafkaConsumer __all__ = [ 'SimpleConsumer', 'MultiProcessConsumer', 'KafkaConsumer' diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index afb8f52a1..7d58b7caa 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -29,7 +29,7 @@ ConsumerRecord = collections.namedtuple("ConsumerRecord", ["topic", "partition", "offset", "timestamp", "timestamp_type", - "key", "value", "checksum", "serialized_key_size", "serialized_value_size"]) + "key", "value", "headers", "checksum", "serialized_key_size", "serialized_value_size", "serialized_header_size"]) CompletedFetch = collections.namedtuple("CompletedFetch", @@ -326,9 +326,6 @@ def fetched_records(self, max_records=None): max_records = self.config['max_poll_records'] assert max_records > 0 - if self._subscriptions.needs_partition_assignment: - return {}, False - drained = collections.defaultdict(list) records_remaining = max_records @@ -375,11 +372,6 @@ def _append(self, drained, part, max_records): tp, next_offset) for record in part_records: - # Fetched compressed messages may include additional records - if record.offset < fetch_offset: - log.debug("Skipping message offset: %s (expecting %s)", - record.offset, fetch_offset) - continue drained[tp].append(record) self._subscriptions.assignment[tp].position = next_offset @@ -397,9 +389,6 @@ def _append(self, drained, part, max_records): def _message_generator(self): """Iterate over fetched_records""" - if self._subscriptions.needs_partition_assignment: - raise StopIteration('Subscription needs partition assignment') - while self._next_partition_records or self._completed_fetches: if not self._next_partition_records: @@ -467,10 +456,12 @@ def _unpack_message_set(self, tp, records): value = self._deserialize( self.config['value_deserializer'], tp.topic, record.value) + headers = record.headers + header_size = sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1 yield ConsumerRecord( tp.topic, tp.partition, record.offset, record.timestamp, - record.timestamp_type, key, value, record.checksum, - key_size, value_size) + record.timestamp_type, key, value, headers, record.checksum, + key_size, value_size, header_size) batch = records.next_batch() @@ -621,7 +612,7 @@ def _handle_offset_response(self, future, response): future.failure(error_type(partition)) return elif error_type is Errors.UnknownTopicOrPartitionError: - log.warn("Received unknown topic or partition error in ListOffset " + log.warning("Received unknown topic or partition error in ListOffset " "request for partition %s. The topic/partition " + "may not exist or the user may not have Describe access " "to it.", partition) @@ -637,9 +628,12 @@ def _handle_offset_response(self, future, response): def _fetchable_partitions(self): fetchable = self._subscriptions.fetchable_partitions() - if self._next_partition_records: - fetchable.discard(self._next_partition_records.topic_partition) - for fetch in self._completed_fetches: + # do not fetch a partition if we have a pending fetch response to process + current = self._next_partition_records + pending = copy.copy(self._completed_fetches) + if current: + fetchable.discard(current.topic_partition) + for fetch in pending: fetchable.discard(fetch.topic_partition) return fetchable @@ -829,10 +823,10 @@ def _parse_fetched_data(self, completed_fetch): raise Errors.OffsetOutOfRangeError({tp: fetch_offset}) elif error_type is Errors.TopicAuthorizationFailedError: - log.warn("Not authorized to read from topic %s.", tp.topic) + log.warning("Not authorized to read from topic %s.", tp.topic) raise Errors.TopicAuthorizationFailedError(set(tp.topic)) elif error_type is Errors.UnknownError: - log.warn("Unknown error fetching data for topic-partition %s", tp) + log.warning("Unknown error fetching data for topic-partition %s", tp) else: raise error_type('Unexpected error while fetching data') @@ -841,12 +835,26 @@ def _parse_fetched_data(self, completed_fetch): return parsed_records - class PartitionRecords(six.Iterator): + class PartitionRecords(object): def __init__(self, fetch_offset, tp, messages): self.fetch_offset = fetch_offset self.topic_partition = tp self.messages = messages - self.message_idx = 0 + # When fetching an offset that is in the middle of a + # compressed batch, we will get all messages in the batch. + # But we want to start 'take' at the fetch_offset + # (or the next highest offset in case the message was compacted) + for i, msg in enumerate(messages): + if msg.offset < fetch_offset: + log.debug("Skipping message offset: %s (expecting %s)", + msg.offset, fetch_offset) + else: + self.message_idx = i + break + + else: + self.message_idx = 0 + self.messages = None # For truthiness evaluation we need to define __len__ or __nonzero__ def __len__(self): @@ -865,8 +873,9 @@ def take(self, n=None): next_idx = self.message_idx + n res = self.messages[self.message_idx:next_idx] self.message_idx = next_idx - if len(self) > 0: - self.fetch_offset = self.messages[self.message_idx].offset + # fetch_offset should be incremented by 1 to parallel the + # subscription position (also incremented by 1) + self.fetch_offset = max(self.fetch_offset, res[-1].offset + 1) return res diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 1ac7e3009..957cffdda 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -87,7 +87,7 @@ class KafkaConsumer(six.Iterator): 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. + Default: 305000. retry_backoff_ms (int): Milliseconds to backoff when retrying on errors. Default: 100. reconnect_backoff_ms (int): The amount of time in milliseconds to @@ -211,11 +211,15 @@ class KafkaConsumer(six.Iterator): (0, 8, 0) enables basic functionality but requires manual partition assignment and offset management. - For the full list of supported versions, see - KafkaClient.API_VERSIONS. Default: None + Default: None api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker api version. Only applies if api_version set to 'auto' + connections_max_idle_ms: Close idle connections after the number of + milliseconds specified by this config. The broker closes idle + connections after connections.max.idle.ms, so this avoids hitting + unexpected socket disconnected errors on the client. + Default: 540000 metric_reporters (list): A list of classes to use as metrics reporters. Implementing the AbstractMetricsReporter interface allows plugging in classes that will be notified of new metric creation. Default: [] @@ -239,6 +243,8 @@ class KafkaConsumer(six.Iterator): Default: None sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' + sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI + sasl mechanism handshake. Default: one of bootstrap servers Note: Configuration parameters are described in more detail at @@ -298,7 +304,8 @@ class KafkaConsumer(six.Iterator): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, - 'sasl_kerberos_service_name': 'kafka' + 'sasl_kerberos_service_name': 'kafka', + 'sasl_kerberos_domain_name': None } DEFAULT_SESSION_TIMEOUT_MS_0_9 = 30000 @@ -603,6 +610,8 @@ def poll(self, timeout_ms=0, max_records=None): assert timeout_ms >= 0, 'Timeout must not be negative' if max_records is None: max_records = self.config['max_poll_records'] + assert isinstance(max_records, int), 'max_records must be an integer' + assert max_records > 0, 'max_records must be positive' # Poll for new data until the timeout expires start = time.time() @@ -650,8 +659,13 @@ def _poll_once(self, timeout_ms, max_records): # Send any new fetches (won't resend pending fetches) self._fetcher.send_fetches() - timeout_ms = min(timeout_ms, self._coordinator.time_to_next_poll()) + timeout_ms = min(timeout_ms, self._coordinator.time_to_next_poll() * 1000) self._client.poll(timeout_ms=timeout_ms) + # after the long poll, we should check whether the group needs to rebalance + # prior to returning data so that the group can stabilize faster + if self._coordinator.need_rejoin(): + return {} + records, _ = self._fetcher.fetched_records(max_records) return records @@ -1060,9 +1074,14 @@ def _message_generator(self): poll_ms = 1000 * (self._consumer_timeout - time.time()) if not self._fetcher.in_flight_fetches(): - poll_ms = 0 + poll_ms = min(poll_ms, self.config['reconnect_backoff_ms']) self._client.poll(timeout_ms=poll_ms) + # after the long poll, we should check whether the group needs to rebalance + # prior to returning data so that the group can stabilize faster + if self._coordinator.need_rejoin(): + continue + # We need to make sure we at least keep up with scheduled tasks, # like heartbeats, auto-commits, and metadata refreshes timeout_at = self._next_timeout() diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 58e3e07bd..758bb92f8 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -8,15 +8,15 @@ from kafka.vendor.six.moves import queue # pylint: disable=import-error -from ..common import KafkaError -from .base import ( +from kafka.errors import KafkaError +from kafka.consumer.base import ( Consumer, AUTO_COMMIT_MSG_COUNT, AUTO_COMMIT_INTERVAL, NO_MESSAGES_WAIT_TIME_SECONDS, FULL_QUEUE_WAIT_TIME_SECONDS, MAX_BACKOFF_SECONDS, ) -from .simple import SimpleConsumer +from kafka.consumer.simple import SimpleConsumer log = logging.getLogger(__name__) @@ -92,7 +92,7 @@ def _mp_consume(client, group, topic, message_queue, size, events, **consumer_op except KafkaError as e: # Retry with exponential backoff - log.error("Problem communicating with Kafka (%s), retrying in %d seconds..." % (e, interval)) + log.exception("Problem communicating with Kafka, retrying in %d seconds...", interval) time.sleep(interval) interval = interval*2 if interval*2 < MAX_BACKOFF_SECONDS else MAX_BACKOFF_SECONDS diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index f5b6a99cf..b60a5865b 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -12,7 +12,7 @@ from kafka.vendor import six from kafka.vendor.six.moves import queue # pylint: disable=import-error -from .base import ( +from kafka.consumer.base import ( Consumer, FETCH_DEFAULT_BLOCK_TIMEOUT, AUTO_COMMIT_MSG_COUNT, @@ -24,13 +24,13 @@ ITER_TIMEOUT_SECONDS, NO_MESSAGES_WAIT_TIME_SECONDS ) -from ..common import ( - FetchRequestPayload, KafkaError, OffsetRequestPayload, - ConsumerFetchSizeTooSmall, +from kafka.errors import ( + KafkaError, ConsumerFetchSizeTooSmall, UnknownTopicOrPartitionError, NotLeaderForPartitionError, OffsetOutOfRangeError, FailedPayloadsError, check_error ) from kafka.protocol.message import PartialMessage +from kafka.structs import FetchRequestPayload, OffsetRequestPayload log = logging.getLogger(__name__) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 3d4dfef1b..10d722ec5 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -68,7 +68,6 @@ def __init__(self, offset_reset_strategy='earliest'): 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 @@ -172,7 +171,6 @@ def change_subscription(self, topics): 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()): @@ -192,12 +190,12 @@ def group_subscribe(self, topics): raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) self._group_subscription.update(topics) - def mark_for_reassignment(self): + def reset_group_subscription(self): + """Reset the group's subscription to only contain topics subscribed by this consumer.""" if self._user_assignment: raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) assert self.subscription is not None, 'Subscription required' 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. @@ -220,18 +218,17 @@ def assign_from_user(self, partitions): if self.subscription is not None: raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) - self._user_assignment.clear() - self._user_assignment.update(partitions) + if self._user_assignment != set(partitions): + self._user_assignment = set(partitions) - for partition in partitions: - if partition not in self.assignment: - self._add_assigned_partition(partition) + 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] + for tp in set(self.assignment.keys()) - self._user_assignment: + del self.assignment[tp] - self.needs_partition_assignment = False - self.needs_fetch_committed_offsets = True + self.needs_fetch_committed_offsets = True def assign_from_subscribed(self, assignments): """Update the assignment to the specified partitions @@ -245,16 +242,18 @@ def assign_from_subscribed(self, assignments): assignments (list of TopicPartition): partitions to assign to this consumer instance. """ - if self.subscription is None: + if not self.partitions_auto_assigned(): 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." % str(tp)) + + # after rebalancing, we always reinitialize the assignment state self.assignment.clear() for tp in assignments: self._add_assigned_partition(tp) - self.needs_partition_assignment = False + self.needs_fetch_committed_offsets = True log.info("Updated partition assignment: %s", assignments) def unsubscribe(self): @@ -262,7 +261,6 @@ def unsubscribe(self): self.subscription = None self._user_assignment.clear() self.assignment.clear() - self.needs_partition_assignment = True self.subscribed_pattern = None def group_subscription(self): diff --git a/kafka/coordinator/assignors/range.py b/kafka/coordinator/assignors/range.py index cbf411e5c..c232d9e41 100644 --- a/kafka/coordinator/assignors/range.py +++ b/kafka/coordinator/assignors/range.py @@ -5,8 +5,8 @@ from kafka.vendor import six -from .abstract import AbstractPartitionAssignor -from ..protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment +from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment log = logging.getLogger(__name__) diff --git a/kafka/coordinator/assignors/roundrobin.py b/kafka/coordinator/assignors/roundrobin.py index c24168599..2d24a5c8b 100644 --- a/kafka/coordinator/assignors/roundrobin.py +++ b/kafka/coordinator/assignors/roundrobin.py @@ -6,9 +6,9 @@ from kafka.vendor import six -from .abstract import AbstractPartitionAssignor -from ...common import TopicPartition -from ..protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment +from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment +from kafka.structs import TopicPartition log = logging.getLogger(__name__) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 301c06d67..7deeaf05d 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -10,13 +10,13 @@ from kafka.vendor import six -from .heartbeat import Heartbeat -from .. import errors as Errors -from ..future import Future -from ..metrics import AnonMeasurable -from ..metrics.stats import Avg, Count, Max, Rate -from ..protocol.commit import GroupCoordinatorRequest, OffsetCommitRequest -from ..protocol.group import (HeartbeatRequest, JoinGroupRequest, +from kafka.coordinator.heartbeat import Heartbeat +from kafka import errors as Errors +from kafka.future import Future +from kafka.metrics import AnonMeasurable +from kafka.metrics.stats import Avg, Count, Max, Rate +from kafka.protocol.commit import GroupCoordinatorRequest, OffsetCommitRequest +from kafka.protocol.group import (HeartbeatRequest, JoinGroupRequest, LeaveGroupRequest, SyncGroupRequest) log = logging.getLogger('kafka.coordinator') @@ -231,20 +231,19 @@ def coordinator(self): Returns: the current coordinator id or None if it is unknown """ - with self._lock: - if self.coordinator_id is None: - return None - elif self._client.is_disconnected(self.coordinator_id): - self.coordinator_dead('Node Disconnected') - return None - else: - return self.coordinator_id + if self.coordinator_id is None: + return None + elif self._client.is_disconnected(self.coordinator_id): + self.coordinator_dead('Node Disconnected') + return None + else: + return self.coordinator_id def ensure_coordinator_ready(self): """Block until the coordinator for this group is known (and we have an active connection -- java client uses unsent queue). """ - with self._lock: + with self._client._lock, self._lock: while self.coordinator_unknown(): # Prior to 0.8.2 there was no group coordinator @@ -274,17 +273,18 @@ def _reset_find_coordinator_future(self, result): self._find_coordinator_future = None def lookup_coordinator(self): - if self._find_coordinator_future is not None: - return self._find_coordinator_future - - # If there is an error sending the group coordinator request - # then _reset_find_coordinator_future will immediately fire and - # set _find_coordinator_future = None - # To avoid returning None, we capture the future in a local variable - self._find_coordinator_future = self._send_group_coordinator_request() - future = self._find_coordinator_future - self._find_coordinator_future.add_both(self._reset_find_coordinator_future) - return future + with self._client._lock, self._lock: + if self._find_coordinator_future is not None: + return self._find_coordinator_future + + # If there is an error sending the group coordinator request + # then _reset_find_coordinator_future will immediately fire and + # set _find_coordinator_future = None + # To avoid returning None, we capture the future in a local variable + future = self._send_group_coordinator_request() + self._find_coordinator_future = future + self._find_coordinator_future.add_both(self._reset_find_coordinator_future) + return future def need_rejoin(self): """Check whether the group should be rejoined (e.g. if metadata changes) @@ -314,6 +314,10 @@ def poll_heartbeat(self): cause = self._heartbeat_thread.failed self._heartbeat_thread = None raise cause # pylint: disable-msg=raising-bad-type + + # Awake the heartbeat thread if needed + if self.heartbeat.should_heartbeat(): + self._lock.notify() self.heartbeat.poll() def time_to_next_heartbeat(self): @@ -344,23 +348,25 @@ def _handle_join_failure(self, _): def ensure_active_group(self): """Ensure that the group is active (i.e. joined and synced)""" with self._lock: - if not self.need_rejoin(): - return - - # call on_join_prepare if needed. We set a flag to make sure that - # we do not call it a second time if the client is woken up before - # a pending rebalance completes. - if not self.rejoining: - self._on_join_prepare(self._generation.generation_id, - self._generation.member_id) - self.rejoining = True - if self._heartbeat_thread is None: self._start_heartbeat_thread() while self.need_rejoin(): self.ensure_coordinator_ready() + # call on_join_prepare if needed. We set a flag + # to make sure that we do not call it a second + # time if the client is woken up before a pending + # rebalance completes. This must be called on each + # iteration of the loop because an event requiring + # a rebalance (such as a metadata refresh which + # changes the matched subscription set) can occur + # while another rebalance is still in progress. + if not self.rejoining: + self._on_join_prepare(self._generation.generation_id, + self._generation.member_id) + self.rejoining = True + # ensure that there are no pending requests to the coordinator. # This is important in particular to avoid resending a pending # JoinGroup request. @@ -481,7 +487,7 @@ def _handle_join_group_response(self, future, send_time, response): log.debug("Received successful JoinGroup response for group %s: %s", self.group_id, response) self.sensors.join_latency.record((time.time() - send_time) * 1000) - with self._lock: + with self._client._lock, self._lock: if self.state is not MemberState.REBALANCING: # if the consumer was woken up before a rebalance completes, # we may have already left the group. In this case, we do @@ -657,7 +663,7 @@ def _handle_group_coordinator_response(self, future, response): error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - with self._lock: + with self._client._lock, self._lock: ok = self._client.cluster.add_group_coordinator(self.group_id, response) if not ok: # This could happen if coordinator metadata is different @@ -687,11 +693,10 @@ def _handle_group_coordinator_response(self, future, response): def coordinator_dead(self, error): """Mark the current coordinator as dead.""" - with self._lock: - if self.coordinator_id is not None: - log.warning("Marking the coordinator dead (node %s) for group %s: %s.", - self.coordinator_id, self.group_id, error) - self.coordinator_id = None + if self.coordinator_id is not None: + log.warning("Marking the coordinator dead (node %s) for group %s: %s.", + self.coordinator_id, self.group_id, error) + self.coordinator_id = None def generation(self): """Get the current generation state if the group is stable. @@ -735,13 +740,13 @@ def __del__(self): def close(self): """Close the coordinator, leave the current group, and reset local generation / member_id""" - with self._lock: + with self._client._lock, self._lock: self._close_heartbeat_thread() self.maybe_leave_group() def maybe_leave_group(self): """Leave the current group and reset local generation/memberId.""" - with self._lock: + with self._client._lock, self._lock: if (not self.coordinator_unknown() and self.state is not MemberState.UNJOINED and self._generation is not Generation.NO_GENERATION): @@ -904,11 +909,10 @@ def close(self): def run(self): try: + log.debug('Heartbeat thread started') while not self.closed: self._run_once() - log.debug('Heartbeat thread closed') - except ReferenceError: log.debug('Heartbeat thread closed due to coordinator gc') @@ -917,6 +921,9 @@ def run(self): self.coordinator.group_id, e) self.failed = e + finally: + log.debug('Heartbeat thread closed') + def _run_once(self): with self.coordinator._lock: if not self.enabled: @@ -933,36 +940,46 @@ def _run_once(self): self.disable() return - # TODO: When consumer.wakeup() is implemented, we need to - # disable here to prevent propagating an exception to this - # heartbeat thread - self.coordinator._client.poll(timeout_ms=0) - - if self.coordinator.coordinator_unknown(): - if not self.coordinator.lookup_coordinator().is_done: + # TODO: When consumer.wakeup() is implemented, we need to + # disable here to prevent propagating an exception to this + # heartbeat thread + # + # Release coordinator lock during client poll to avoid deadlocks + # if/when connection errback needs coordinator lock + self.coordinator._client.poll(timeout_ms=0) + + if self.coordinator.coordinator_unknown(): + future = self.coordinator.lookup_coordinator() + if not future.is_done or future.failed(): + # the immediate future check ensures that we backoff + # properly in the case that no brokers are available + # to connect to (and the future is automatically failed). + with self.coordinator._lock: self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) - elif self.coordinator.heartbeat.session_timeout_expired(): - # the session timeout has expired without seeing a - # successful heartbeat, so we should probably make sure - # the coordinator is still healthy. - log.debug('Heartbeat session expired, marking coordinator dead') - self.coordinator.coordinator_dead('Heartbeat session expired') - - elif self.coordinator.heartbeat.poll_timeout_expired(): - # the poll timeout has expired, which means that the - # foreground thread has stalled in between calls to - # poll(), so we explicitly leave the group. - log.debug('Heartbeat poll expired, leaving group') - self.coordinator.maybe_leave_group() - - elif not self.coordinator.heartbeat.should_heartbeat(): - # poll again after waiting for the retry backoff in case - # the heartbeat failed or the coordinator disconnected - log.debug('Not ready to heartbeat, waiting') + elif self.coordinator.heartbeat.session_timeout_expired(): + # the session timeout has expired without seeing a + # successful heartbeat, so we should probably make sure + # the coordinator is still healthy. + log.warning('Heartbeat session expired, marking coordinator dead') + self.coordinator.coordinator_dead('Heartbeat session expired') + + elif self.coordinator.heartbeat.poll_timeout_expired(): + # the poll timeout has expired, which means that the + # foreground thread has stalled in between calls to + # poll(), so we explicitly leave the group. + log.warning('Heartbeat poll expired, leaving group') + self.coordinator.maybe_leave_group() + + elif not self.coordinator.heartbeat.should_heartbeat(): + # poll again after waiting for the retry backoff in case + # the heartbeat failed or the coordinator disconnected + log.log(0, 'Not ready to heartbeat, waiting') + with self.coordinator._lock: self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) - else: + else: + with self.coordinator._client._lock, self.coordinator._lock: self.coordinator.heartbeat.sent_heartbeat() future = self.coordinator._send_heartbeat_request() future.add_callback(self._handle_heartbeat_success) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index ab30883c8..647a6b585 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -7,17 +7,17 @@ from kafka.vendor import six -from .base import BaseCoordinator, Generation -from .assignors.range import RangePartitionAssignor -from .assignors.roundrobin import RoundRobinPartitionAssignor -from .protocol import ConsumerProtocol -from .. import errors as Errors -from ..future import Future -from ..metrics import AnonMeasurable -from ..metrics.stats import Avg, Count, Max, Rate -from ..protocol.commit import OffsetCommitRequest, OffsetFetchRequest -from ..structs import OffsetAndMetadata, TopicPartition -from ..util import WeakMethod +from kafka.coordinator.base import BaseCoordinator, Generation +from kafka.coordinator.assignors.range import RangePartitionAssignor +from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor +from kafka.coordinator.protocol import ConsumerProtocol +import kafka.errors as Errors +from kafka.future import Future +from kafka.metrics import AnonMeasurable +from kafka.metrics.stats import Avg, Count, Max, Rate +from kafka.protocol.commit import OffsetCommitRequest, OffsetFetchRequest +from kafka.structs import OffsetAndMetadata, TopicPartition +from kafka.util import WeakMethod log = logging.getLogger(__name__) @@ -84,6 +84,8 @@ def __init__(self, client, subscription, metrics, **configs): self.config[key] = configs[key] self._subscription = subscription + self._is_leader = False + self._joined_subscription = set() self._metadata_snapshot = self._build_metadata_snapshot(subscription, client.cluster) self._assignment_snapshot = None self._cluster = client.cluster @@ -132,11 +134,22 @@ def protocol_type(self): def group_protocols(self): """Returns list of preferred (protocols, metadata)""" - topics = self._subscription.subscription - assert topics is not None, 'Consumer has not subscribed to topics' + if self._subscription.subscription is None: + raise Errors.IllegalStateError('Consumer has not subscribed to topics') + # dpkp note: I really dislike this. + # why? because we are using this strange method group_protocols, + # which is seemingly innocuous, to set internal state (_joined_subscription) + # that is later used to check whether metadata has changed since we joined a group + # but there is no guarantee that this method, group_protocols, will get called + # in the correct sequence or that it will only be called when we want it to be. + # So this really should be moved elsewhere, but I don't have the energy to + # work that out right now. If you read this at some later date after the mutable + # state has bitten you... I'm sorry! It mimics the java client, and that's the + # best I've got for now. + self._joined_subscription = set(self._subscription.subscription) metadata_list = [] for assignor in self.config['assignors']: - metadata = assignor.metadata(topics) + metadata = assignor.metadata(self._joined_subscription) group_protocol = (assignor.name, metadata) metadata_list.append(group_protocol) return metadata_list @@ -158,21 +171,29 @@ def _handle_metadata_update(self, cluster): # check if there are any changes to the metadata which should trigger # a rebalance - if self._subscription_metadata_changed(cluster): - - if (self.config['api_version'] >= (0, 9) - and self.config['group_id'] is not None): - - 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._metadata_snapshot[topic] - ]) + if self._subscription.partitions_auto_assigned(): + metadata_snapshot = self._build_metadata_snapshot(self._subscription, cluster) + if self._metadata_snapshot != metadata_snapshot: + self._metadata_snapshot = metadata_snapshot + + # If we haven't got group coordinator support, + # just assign all partitions locally + if self._auto_assign_all_partitions(): + self._subscription.assign_from_subscribed([ + TopicPartition(topic, partition) + for topic in self._subscription.subscription + for partition in self._metadata_snapshot[topic] + ]) + + def _auto_assign_all_partitions(self): + # For users that use "subscribe" without group support, + # we will simply assign all partitions to this consumer + if self.config['api_version'] < (0, 9): + return True + elif self.config['group_id'] is None: + return True + else: + return False def _build_metadata_snapshot(self, subscription, cluster): metadata_snapshot = {} @@ -181,16 +202,6 @@ def _build_metadata_snapshot(self, subscription, cluster): metadata_snapshot[topic] = set(partitions) return metadata_snapshot - def _subscription_metadata_changed(self, cluster): - if not self._subscription.partitions_auto_assigned(): - return False - - metadata_snapshot = self._build_metadata_snapshot(self._subscription, cluster) - if self._metadata_snapshot != metadata_snapshot: - self._metadata_snapshot = metadata_snapshot - return True - return False - def _lookup_assignor(self, name): for assignor in self.config['assignors']: if assignor.name == name: @@ -199,12 +210,10 @@ def _lookup_assignor(self, name): def _on_join_complete(self, generation, member_id, protocol, member_assignment_bytes): - # if we were the assignor, then we need to make sure that there have - # been no metadata updates since the rebalance begin. Otherwise, we - # won't rebalance again until the next metadata change - if self._assignment_snapshot is not None and self._assignment_snapshot != self._metadata_snapshot: - self._subscription.mark_for_reassignment() - return + # only the leader is responsible for monitoring for metadata changes + # (i.e. partition changes) + if not self._is_leader: + self._assignment_snapshot = None assignor = self._lookup_assignor(protocol) assert assignor, 'Coordinator selected invalid assignment protocol: %s' % protocol @@ -307,6 +316,7 @@ def _perform_assignment(self, leader_id, assignment_strategy, members): # keep track of the metadata used for assignment so that we can check # after rebalance completion whether anything has changed self._cluster.request_update() + self._is_leader = True self._assignment_snapshot = self._metadata_snapshot log.debug("Performing assignment for group %s using strategy %s" @@ -338,8 +348,8 @@ def _on_join_prepare(self, generation, member_id): " for group %s failed on_partitions_revoked", self._subscription.listener, self.group_id) - self._assignment_snapshot = None - self._subscription.mark_for_reassignment() + self._is_leader = False + self._subscription.reset_group_subscription() def need_rejoin(self): """Check whether the group should be rejoined @@ -347,9 +357,23 @@ def need_rejoin(self): 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)) + if not self._subscription.partitions_auto_assigned(): + return False + + if self._auto_assign_all_partitions(): + return False + + # we need to rejoin if we performed the assignment and metadata has changed + if (self._assignment_snapshot is not None + and self._assignment_snapshot != self._metadata_snapshot): + return True + + # we need to join if our subscription has changed since the last join + if (self._joined_subscription is not None + and self._joined_subscription != self._subscription.subscription): + return True + + return super(ConsumerCoordinator, self).need_rejoin() def refresh_committed_offsets_if_needed(self): """Fetch committed offsets for assigned partitions.""" @@ -417,10 +441,13 @@ def commit_offsets_async(self, offsets, callback=None): 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: + kafka.future.Future """ self._invoke_completed_offset_commit_callbacks() if not self.coordinator_unknown(): - self._do_commit_offsets_async(offsets, callback) + future = self._do_commit_offsets_async(offsets, callback) else: # we don't know the current coordinator, so try to find it and then # send the commit or fail (we don't want recursive retries which can @@ -438,7 +465,9 @@ def commit_offsets_async(self, offsets, callback=None): # its completion). Note that commits are treated as heartbeats by the # coordinator, so there is no need to explicitly allow heartbeats # through delayed task execution. - self._client.poll() # no wakeup if we add that feature + self._client.poll(timeout_ms=0) # no wakeup if we add that feature + + return future def _do_commit_offsets_async(self, offsets, callback=None): assert self.config['api_version'] >= (0, 8, 1), 'Unsupported Broker API' @@ -644,17 +673,7 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): log.debug("OffsetCommit for group %s failed: %s", self.group_id, error) self.reset_generation() - future.failure(Errors.CommitFailedError( - "Commit cannot be completed since the group has" - " already rebalanced and assigned the partitions to" - " another member. This means that the time between" - " subsequent calls to poll() was longer than the" - " configured session_timeout_ms, which typically" - " implies that the poll loop is spending too much time" - " message processing. You can address this either by" - " increasing the session timeout or by reducing the" - " maximum size of batches returned in poll() with" - " max_poll_records.")) + future.failure(Errors.CommitFailedError()) return else: log.error("Group %s failed to commit partition %s at offset" diff --git a/kafka/errors.py b/kafka/errors.py index c70853c69..fb9576c3f 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -54,6 +54,10 @@ class StaleMetadata(KafkaError): invalid_metadata = True +class MetadataEmptyBrokerList(KafkaError): + retriable = True + + class UnrecognizedBrokerVersion(KafkaError): pass @@ -447,11 +451,15 @@ def __init__(self, payload, *args): self.payload = payload -class ConnectionError(KafkaError): +class KafkaConnectionError(KafkaError): retriable = True invalid_metadata = True +class ConnectionError(KafkaConnectionError): + """Deprecated""" + + class BufferUnderflowError(KafkaError): pass @@ -468,10 +476,6 @@ class ConsumerNoMoreData(KafkaError): pass -class ConsumerTimeout(KafkaError): - pass - - class ProtocolError(KafkaError): pass @@ -517,13 +521,13 @@ def check_error(response): RETRY_BACKOFF_ERROR_TYPES = ( KafkaUnavailableError, LeaderNotAvailableError, - ConnectionError, FailedPayloadsError + KafkaConnectionError, FailedPayloadsError ) RETRY_REFRESH_ERROR_TYPES = ( NotLeaderForPartitionError, UnknownTopicOrPartitionError, - LeaderNotAvailableError, ConnectionError + LeaderNotAvailableError, KafkaConnectionError ) diff --git a/kafka/metrics/__init__.py b/kafka/metrics/__init__.py index 6055142a6..2a62d6334 100644 --- a/kafka/metrics/__init__.py +++ b/kafka/metrics/__init__.py @@ -1,13 +1,13 @@ from __future__ import absolute_import -from .compound_stat import NamedMeasurable -from .dict_reporter import DictReporter -from .kafka_metric import KafkaMetric -from .measurable import AnonMeasurable -from .metric_config import MetricConfig -from .metric_name import MetricName -from .metrics import Metrics -from .quota import Quota +from kafka.metrics.compound_stat import NamedMeasurable +from kafka.metrics.dict_reporter import DictReporter +from kafka.metrics.kafka_metric import KafkaMetric +from kafka.metrics.measurable import AnonMeasurable +from kafka.metrics.metric_config import MetricConfig +from kafka.metrics.metric_name import MetricName +from kafka.metrics.metrics import Metrics +from kafka.metrics.quota import Quota __all__ = [ 'AnonMeasurable', 'DictReporter', 'KafkaMetric', 'MetricConfig', diff --git a/kafka/metrics/metrics.py b/kafka/metrics/metrics.py index e9c465deb..f2e99edc9 100644 --- a/kafka/metrics/metrics.py +++ b/kafka/metrics/metrics.py @@ -257,3 +257,5 @@ def close(self): """Close this metrics repository.""" for reporter in self._reporters: reporter.close() + + self._metrics.clear() diff --git a/kafka/metrics/stats/__init__.py b/kafka/metrics/stats/__init__.py index ab1fb715f..a3d535dfd 100644 --- a/kafka/metrics/stats/__init__.py +++ b/kafka/metrics/stats/__init__.py @@ -1,15 +1,15 @@ from __future__ import absolute_import -from .avg import Avg -from .count import Count -from .histogram import Histogram -from .max_stat import Max -from .min_stat import Min -from .percentile import Percentile -from .percentiles import Percentiles -from .rate import Rate -from .sensor import Sensor -from .total import Total +from kafka.metrics.stats.avg import Avg +from kafka.metrics.stats.count import Count +from kafka.metrics.stats.histogram import Histogram +from kafka.metrics.stats.max_stat import Max +from kafka.metrics.stats.min_stat import Min +from kafka.metrics.stats.percentile import Percentile +from kafka.metrics.stats.percentiles import Percentiles +from kafka.metrics.stats.rate import Rate +from kafka.metrics.stats.sensor import Sensor +from kafka.metrics.stats.total import Total __all__ = [ 'Avg', 'Count', 'Histogram', 'Max', 'Min', 'Percentile', 'Percentiles', diff --git a/kafka/partitioner/__init__.py b/kafka/partitioner/__init__.py index 299b485d9..a9dbbdccb 100644 --- a/kafka/partitioner/__init__.py +++ b/kafka/partitioner/__init__.py @@ -1,8 +1,8 @@ from __future__ import absolute_import -from .default import DefaultPartitioner -from .hashed import HashedPartitioner, Murmur2Partitioner, LegacyPartitioner -from .roundrobin import RoundRobinPartitioner +from kafka.partitioner.default import DefaultPartitioner +from kafka.partitioner.hashed import HashedPartitioner, Murmur2Partitioner, LegacyPartitioner +from kafka.partitioner.roundrobin import RoundRobinPartitioner __all__ = [ 'DefaultPartitioner', 'RoundRobinPartitioner', 'HashedPartitioner', diff --git a/kafka/partitioner/default.py b/kafka/partitioner/default.py index 087166c0f..e4d9df5dc 100644 --- a/kafka/partitioner/default.py +++ b/kafka/partitioner/default.py @@ -2,7 +2,7 @@ import random -from .hashed import murmur2 +from kafka.partitioner.hashed import murmur2 class DefaultPartitioner(object): diff --git a/kafka/partitioner/hashed.py b/kafka/partitioner/hashed.py index 06307f08d..be92daffa 100644 --- a/kafka/partitioner/hashed.py +++ b/kafka/partitioner/hashed.py @@ -2,7 +2,7 @@ from kafka.vendor import six -from .base import Partitioner +from kafka.partitioner.base import Partitioner class Murmur2Partitioner(Partitioner): diff --git a/kafka/partitioner/roundrobin.py b/kafka/partitioner/roundrobin.py index 9ac2ed0cd..e68c37242 100644 --- a/kafka/partitioner/roundrobin.py +++ b/kafka/partitioner/roundrobin.py @@ -1,6 +1,6 @@ from __future__ import absolute_import -from .base import Partitioner +from kafka.partitioner.base import Partitioner class RoundRobinPartitioner(Partitioner): diff --git a/kafka/producer/__init__.py b/kafka/producer/__init__.py index 5213fe818..54fd8d2ae 100644 --- a/kafka/producer/__init__.py +++ b/kafka/producer/__init__.py @@ -1,8 +1,8 @@ from __future__ import absolute_import -from .kafka import KafkaProducer -from .simple import SimpleProducer -from .keyed import KeyedProducer +from kafka.producer.kafka import KafkaProducer +from kafka.producer.simple import SimpleProducer +from kafka.producer.keyed import KeyedProducer __all__ = [ 'KafkaProducer', diff --git a/kafka/producer/base.py b/kafka/producer/base.py index c038bd3a0..956cef6c5 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -14,13 +14,13 @@ from kafka.vendor import six -from kafka.structs import ( - ProduceRequestPayload, ProduceResponsePayload, TopicPartition, RetryOptions) from kafka.errors import ( 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.structs import ( + ProduceRequestPayload, ProduceResponsePayload, TopicPartition, RetryOptions) log = logging.getLogger('kafka.producer') @@ -226,7 +226,7 @@ class Producer(object): Arguments: client (kafka.SimpleClient): instance to use for broker - communications. If async=True, the background thread will use + communications. If async_send=True, the background thread will use :meth:`client.copy`, which is expected to return a thread-safe object. codec (kafka.protocol.ALL_CODECS): compression codec to use. @@ -238,11 +238,11 @@ class Producer(object): sync_fail_on_error (bool, optional): whether sync producer should raise exceptions (True), or just return errors (False), defaults to True. - async (bool, optional): send message using a background thread, + async_send (bool, optional): send message using a background thread, defaults to False. - batch_send_every_n (int, optional): If async is True, messages are + batch_send_every_n (int, optional): If async_send is True, messages are sent in batches of this size, defaults to 20. - batch_send_every_t (int or float, optional): If async is True, + batch_send_every_t (int or float, optional): If async_send is True, messages are sent immediately after this timeout in seconds, even if there are fewer than batch_send_every_n, defaults to 20. async_retry_limit (int, optional): number of retries for failed messages @@ -268,8 +268,10 @@ class Producer(object): defaults to 30. Deprecated Arguments: + async (bool, optional): send message using a background thread, + defaults to False. Deprecated, use 'async_send' batch_send (bool, optional): If True, messages are sent by a background - thread in batches, defaults to False. Deprecated, use 'async' + thread in batches, defaults to False. Deprecated, use 'async_send' """ ACK_NOT_REQUIRED = 0 # No ack is required ACK_AFTER_LOCAL_WRITE = 1 # Send response after it is written to log @@ -282,8 +284,8 @@ def __init__(self, client, codec=None, codec_compresslevel=None, sync_fail_on_error=SYNC_FAIL_ON_ERROR_DEFAULT, - async=False, - batch_send=False, # deprecated, use async + async_send=False, + batch_send=False, # deprecated, use async_send batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, async_retry_limit=ASYNC_RETRY_LIMIT, @@ -292,15 +294,21 @@ def __init__(self, client, async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT, async_log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR, - async_stop_timeout=ASYNC_STOP_TIMEOUT_SECS): + async_stop_timeout=ASYNC_STOP_TIMEOUT_SECS, + **kwargs): + + # async renamed async_send for python3.7 support + if 'async' in kwargs: + log.warning('Deprecated async option found -- use async_send') + async_send = kwargs['async'] - if async: + if async_send: assert batch_send_every_n > 0 assert batch_send_every_t > 0 assert async_queue_maxsize >= 0 self.client = client - self.async = async + self.async_send = async_send self.req_acks = req_acks self.ack_timeout = ack_timeout self.stopped = False @@ -313,7 +321,7 @@ def __init__(self, client, self.codec = codec self.codec_compresslevel = codec_compresslevel - if self.async: + if self.async_send: # Messages are sent through this queue self.queue = Queue(async_queue_maxsize) self.async_queue_put_timeout = async_queue_put_timeout @@ -364,7 +372,6 @@ def send_messages(self, topic, partition, *msg): 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 @@ -400,7 +407,7 @@ def _send_messages(self, topic, partition, *msg, **kwargs): if key is not None and not isinstance(key, six.binary_type): raise TypeError("the key must be type bytes") - if self.async: + if self.async_send: for idx, m in enumerate(msg): try: item = (TopicPartition(topic, partition), m, key) @@ -435,7 +442,7 @@ def stop(self, timeout=None): log.warning('timeout argument to stop() is deprecated - ' 'it will be removed in future release') - if not self.async: + if not self.async_send: log.warning('producer.stop() called, but producer is not async') return @@ -443,7 +450,7 @@ def stop(self, timeout=None): log.warning('producer.stop() called, but producer is already stopped') return - if self.async: + if self.async_send: self.queue.put((STOP_ASYNC_PRODUCER, None, None)) self.thread_stop_event.set() self.thread.join() @@ -471,5 +478,5 @@ def stop(self, timeout=None): self.stopped = True def __del__(self): - if self.async and not self.stopped: + if self.async_send and not self.stopped: self.stop() diff --git a/kafka/producer/buffer.py b/kafka/producer/buffer.py index 19ea7322e..8a8d7174c 100644 --- a/kafka/producer/buffer.py +++ b/kafka/producer/buffer.py @@ -5,7 +5,7 @@ import threading import time -from ..metrics.stats import Rate +from kafka.metrics.stats import Rate import kafka.errors as Errors diff --git a/kafka/producer/future.py b/kafka/producer/future.py index e39a0a97a..1c5d6d7bf 100644 --- a/kafka/producer/future.py +++ b/kafka/producer/future.py @@ -3,8 +3,8 @@ import collections import threading -from .. import errors as Errors -from ..future import Future +from kafka import errors as Errors +from kafka.future import Future class FutureProduceResult(Future): @@ -29,11 +29,11 @@ def wait(self, timeout=None): class FutureRecordMetadata(Future): - def __init__(self, produce_future, relative_offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size): + def __init__(self, produce_future, relative_offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size): super(FutureRecordMetadata, self).__init__() self._produce_future = produce_future # packing args as a tuple is a minor speed optimization - self.args = (relative_offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size) + self.args = (relative_offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size) produce_future.add_callback(self._produce_success) produce_future.add_errback(self.failure) @@ -42,7 +42,7 @@ def _produce_success(self, offset_and_timestamp): # Unpacking from args tuple is minor speed optimization (relative_offset, timestamp_ms, checksum, - serialized_key_size, serialized_value_size) = self.args + serialized_key_size, serialized_value_size, serialized_header_size) = self.args # None is when Broker does not support the API (<0.10) and # -1 is when the broker is configured for CREATE_TIME timestamps @@ -53,7 +53,7 @@ def _produce_success(self, offset_and_timestamp): tp = self._produce_future.topic_partition metadata = RecordMetadata(tp[0], tp[1], tp, offset, timestamp_ms, checksum, serialized_key_size, - serialized_value_size) + serialized_value_size, serialized_header_size) self.success(metadata) def get(self, timeout=None): @@ -68,4 +68,4 @@ def get(self, timeout=None): RecordMetadata = collections.namedtuple( 'RecordMetadata', ['topic', 'partition', 'topic_partition', 'offset', 'timestamp', - 'checksum', 'serialized_key_size', 'serialized_value_size']) + 'checksum', 'serialized_key_size', 'serialized_value_size', 'serialized_header_size']) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 32d00200e..9926081de 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -8,20 +8,20 @@ import time import weakref -from ..vendor import six - -from .. import errors as Errors -from ..client_async import KafkaClient, selectors -from ..codec import has_gzip, has_snappy, has_lz4 -from ..metrics import MetricConfig, Metrics -from ..partitioner.default import DefaultPartitioner -from ..record.default_records import DefaultRecordBatchBuilder -from ..record.legacy_records import LegacyRecordBatchBuilder -from ..serializer import Serializer -from ..structs import TopicPartition -from .future import FutureRecordMetadata, FutureProduceResult -from .record_accumulator import AtomicInteger, RecordAccumulator -from .sender import Sender +from kafka.vendor import six + +import kafka.errors as Errors +from kafka.client_async import KafkaClient, selectors +from kafka.codec import has_gzip, has_snappy, has_lz4 +from kafka.metrics import MetricConfig, Metrics +from kafka.partitioner.default import DefaultPartitioner +from kafka.producer.future import FutureRecordMetadata, FutureProduceResult +from kafka.producer.record_accumulator import AtomicInteger, RecordAccumulator +from kafka.producer.sender import Sender +from kafka.record.default_records import DefaultRecordBatchBuilder +from kafka.record.legacy_records import LegacyRecordBatchBuilder +from kafka.serializer import Serializer +from kafka.structs import TopicPartition log = logging.getLogger(__name__) @@ -174,6 +174,11 @@ class KafkaProducer(object): will block up to max_block_ms, raising an exception on timeout. In the current implementation, this setting is an approximation. Default: 33554432 (32MB) + connections_max_idle_ms: Close idle connections after the number of + milliseconds specified by this config. The broker closes idle + connections after connections.max.idle.ms, so this avoids hitting + unexpected socket disconnected errors on the client. + Default: 540000 max_block_ms (int): Number of milliseconds to block during :meth:`~kafka.KafkaProducer.send` and :meth:`~kafka.KafkaProducer.partitions_for`. These methods can be @@ -245,8 +250,7 @@ class KafkaProducer(object): default: none. api_version (tuple): Specify which Kafka API version to use. If set to None, the client will attempt to infer the broker version by probing - various APIs. For a full list of supported versions, see - KafkaClient.API_VERSIONS. Default: None + various APIs. Example: (0, 10, 2). Default: None api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker api version. Only applies if api_version set to 'auto' @@ -269,6 +273,8 @@ class KafkaProducer(object): Default: None sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' + sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI + sasl mechanism handshake. Default: one of bootstrap servers Note: Configuration parameters are described in more detail at @@ -281,6 +287,7 @@ class KafkaProducer(object): 'key_serializer': None, 'value_serializer': None, 'acks': 1, + 'bootstrap_topics_filter': set(), 'compression_type': None, 'retries': 0, 'batch_size': 16384, @@ -318,7 +325,8 @@ class KafkaProducer(object): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, - 'sasl_kerberos_service_name': 'kafka' + 'sasl_kerberos_service_name': 'kafka', + 'sasl_kerberos_domain_name': None } _COMPRESSORS = { @@ -512,7 +520,7 @@ def _estimate_size_in_bytes(self, key, value, headers=[]): return LegacyRecordBatchBuilder.estimate_size_in_bytes( magic, self.config['compression_type'], key, value) - def send(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): """Publish a message to a topic. Arguments: @@ -533,6 +541,8 @@ def send(self, topic, value=None, key=None, partition=None, timestamp_ms=None): partition (but if key is None, partition is chosen randomly). Must be type bytes, or be serializable to bytes via configured key_serializer. + headers (optional): a list of header key value pairs. List items + are tuples of str key and bytes value. timestamp_ms (int, optional): epoch milliseconds (from Jan 1 1970 UTC) to use as the message timestamp. Defaults to current time. @@ -548,8 +558,6 @@ def send(self, topic, value=None, key=None, partition=None, timestamp_ms=None): assert not (value is None and key is None), 'Need at least one: key or value' key_bytes = value_bytes = None try: - # first make sure the metadata for the topic is - # available self._wait_on_metadata(topic, self.config['max_block_ms'] / 1000.0) key_bytes = self._serialize( @@ -558,16 +566,24 @@ def send(self, topic, value=None, key=None, partition=None, timestamp_ms=None): value_bytes = self._serialize( self.config['value_serializer'], topic, value) + assert type(key_bytes) in (bytes, bytearray, memoryview, type(None)) + assert type(value_bytes) in (bytes, bytearray, memoryview, type(None)) + partition = self._partition(topic, partition, key, value, key_bytes, value_bytes) - message_size = self._estimate_size_in_bytes(key, value) + if headers is None: + headers = [] + assert type(headers) == list + assert all(type(item) == tuple and len(item) == 2 and type(item[0]) == str and type(item[1]) == bytes for item in headers) + + message_size = self._estimate_size_in_bytes(key_bytes, value_bytes, headers) self._ensure_valid_record_size(message_size) tp = TopicPartition(topic, partition) - log.debug("Sending (key=%r value=%r) to %s", key, value, tp) + log.debug("Sending (key=%r value=%r headers=%r) to %s", key, value, headers, tp) result = self._accumulator.append(tp, timestamp_ms, - key_bytes, value_bytes, + key_bytes, value_bytes, headers, self.config['max_block_ms'], estimated_size=message_size) future, batch_is_full, new_batch_created = result @@ -586,7 +602,8 @@ def send(self, topic, value=None, key=None, partition=None, timestamp_ms=None): FutureProduceResult(TopicPartition(topic, partition)), -1, None, None, len(key_bytes) if key_bytes is not None else -1, - len(value_bytes) if value_bytes is not None else -1 + len(value_bytes) if value_bytes is not None else -1, + sum(len(h_key.encode("utf-8")) + len(h_value) for h_key, h_value in headers) if headers else -1, ).failure(e) def flush(self, timeout=None): diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 9fba33bbf..62bb733fc 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -3,8 +3,8 @@ import logging import warnings -from .base import Producer -from ..partitioner import HashedPartitioner +from kafka.producer.base import Producer +from kafka.partitioner import HashedPartitioner log = logging.getLogger(__name__) @@ -46,4 +46,4 @@ def send(self, topic, key, msg): return self.send_messages(topic, key, msg) def __repr__(self): - return '' % self.async + return '' % self.async_send diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 5158474f8..84b01d1b5 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -6,12 +6,12 @@ import threading import time -from .. import errors as Errors -from .buffer import SimpleBufferPool -from .future import FutureRecordMetadata, FutureProduceResult -from ..structs import TopicPartition +import kafka.errors as Errors +from kafka.producer.buffer import SimpleBufferPool +from kafka.producer.future import FutureRecordMetadata, FutureProduceResult from kafka.record.memory_records import MemoryRecordsBuilder from kafka.record.legacy_records import LegacyRecordBatchBuilder +from kafka.structs import TopicPartition log = logging.getLogger(__name__) @@ -55,8 +55,8 @@ def __init__(self, tp, records, buffer): def record_count(self): return self.records.next_offset() - def try_append(self, timestamp_ms, key, value): - metadata = self.records.append(timestamp_ms, key, value) + def try_append(self, timestamp_ms, key, value, headers): + metadata = self.records.append(timestamp_ms, key, value, headers) if metadata is None: return None @@ -65,7 +65,8 @@ def try_append(self, timestamp_ms, key, value): future = FutureRecordMetadata(self.produce_future, metadata.offset, metadata.timestamp, metadata.crc, len(key) if key is not None else -1, - len(value) if value is not None else -1) + len(value) if value is not None else -1, + sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1) return future def done(self, base_offset=None, timestamp_ms=None, exception=None): @@ -196,7 +197,7 @@ def __init__(self, **configs): self.muted = set() self._drain_index = 0 - def append(self, tp, timestamp_ms, key, value, max_time_to_block_ms, + def append(self, tp, timestamp_ms, key, value, headers, max_time_to_block_ms, estimated_size=0): """Add a record to the accumulator, return the append result. @@ -209,6 +210,7 @@ def append(self, tp, timestamp_ms, key, value, max_time_to_block_ms, timestamp_ms (int): The timestamp of the record (epoch ms) key (bytes): The key for the record value (bytes): The value for the record + headers (List[Tuple[str, bytes]]): The header fields for the record max_time_to_block_ms (int): The maximum time in milliseconds to block for buffer memory to be available @@ -231,7 +233,7 @@ def append(self, tp, timestamp_ms, key, value, max_time_to_block_ms, dq = self._batches[tp] if dq: last = dq[-1] - future = last.try_append(timestamp_ms, key, value) + future = last.try_append(timestamp_ms, key, value, headers) if future is not None: batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False @@ -246,7 +248,7 @@ def append(self, tp, timestamp_ms, key, value, max_time_to_block_ms, if dq: last = dq[-1] - future = last.try_append(timestamp_ms, key, value) + future = last.try_append(timestamp_ms, key, value, headers) if future is not None: # Somebody else found us a batch, return the one we # waited for! Hopefully this doesn't happen often... @@ -261,7 +263,7 @@ def append(self, tp, timestamp_ms, key, value, max_time_to_block_ms, ) batch = ProducerBatch(tp, records, buf) - future = batch.try_append(timestamp_ms, key, value) + future = batch.try_append(timestamp_ms, key, value, headers) if not future: raise Exception() diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 48ad06e64..895045da6 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -8,12 +8,12 @@ from kafka.vendor import six -from .. import errors as Errors -from ..metrics.measurable import AnonMeasurable -from ..metrics.stats import Avg, Max, Rate -from ..protocol.produce import ProduceRequest -from ..structs import TopicPartition -from ..version import __version__ +from kafka import errors as Errors +from kafka.metrics.measurable import AnonMeasurable +from kafka.metrics.stats import Avg, Max, Rate +from kafka.protocol.produce import ProduceRequest +from kafka.structs import TopicPartition +from kafka.version import __version__ log = logging.getLogger(__name__) diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index 90b3d4a7d..91e0abc4c 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -6,7 +6,7 @@ from kafka.vendor.six.moves import xrange # pylint: disable=import-error -from .base import Producer +from kafka.producer.base import Producer log = logging.getLogger(__name__) @@ -51,4 +51,4 @@ def send_messages(self, topic, *msg): ) def __repr__(self): - return '' % self.async + return '' % self.async_send diff --git a/kafka/protocol/__init__.py b/kafka/protocol/__init__.py index 4dcf4a4eb..050a0854f 100644 --- a/kafka/protocol/__init__.py +++ b/kafka/protocol/__init__.py @@ -1,6 +1,6 @@ from __future__ import absolute_import -from .legacy import ( +from kafka.protocol.legacy import ( create_message, create_gzip_message, create_snappy_message, create_message_set, CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY, ALL_CODECS, @@ -42,4 +42,6 @@ 31: 'DeleteAcls', 32: 'DescribeConfigs', 33: 'AlterConfigs', + 36: 'SaslAuthenticate', + 37: 'CreatePartitions', } diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index 09746bf5e..ed9026a52 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -1,7 +1,7 @@ from __future__ import absolute_import -from .api import Request, Response -from .types import Array, Boolean, Bytes, Int16, Int32, Schema, String +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Schema, String class ApiVersionResponse_v0(Response): @@ -256,7 +256,6 @@ class DescribeGroupsResponse_v1(Response): ) - class DescribeGroupsRequest_v0(Request): API_KEY = 15 API_VERSION = 0 @@ -286,6 +285,12 @@ class SaslHandShakeResponse_v0(Response): ) +class SaslHandShakeResponse_v1(Response): + API_KEY = 17 + API_VERSION = 1 + SCHEMA = SaslHandShakeResponse_v0.SCHEMA + + class SaslHandShakeRequest_v0(Request): API_KEY = 17 API_VERSION = 0 @@ -294,5 +299,162 @@ class SaslHandShakeRequest_v0(Request): ('mechanism', String('utf-8')) ) -SaslHandShakeRequest = [SaslHandShakeRequest_v0] -SaslHandShakeResponse = [SaslHandShakeResponse_v0] + +class SaslHandShakeRequest_v1(Request): + API_KEY = 17 + API_VERSION = 1 + RESPONSE_TYPE = SaslHandShakeResponse_v1 + SCHEMA = SaslHandShakeRequest_v0.SCHEMA + + +SaslHandShakeRequest = [SaslHandShakeRequest_v0, SaslHandShakeRequest_v1] +SaslHandShakeResponse = [SaslHandShakeResponse_v0, SaslHandShakeResponse_v1] + +class AlterConfigsResponse_v0(Response): + API_KEY = 33 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('resources', Array( + ('error_code', Int16), + ('error_message', String('utf-8')), + ('resource_type', Int8), + ('resource_name', String('utf-8')))) + ) + +class AlterConfigsRequest_v0(Request): + API_KEY = 33 + API_VERSION = 0 + RESPONSE_TYPE = AlterConfigsResponse_v0 + SCHEMA = Schema( + ('resources', Array( + ('resource_type', Int8), + ('resource_name', String('utf-8')), + ('config_entries', Array( + ('config_name', String('utf-8')), + ('config_value', String('utf-8')))))), + ('validate_only', Boolean) + ) + +AlterConfigsRequest = [AlterConfigsRequest_v0] +AlterConfigsResponse = [AlterConfigsResponse_v0] + + +class DescribeConfigsResponse_v0(Response): + API_KEY = 32 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('resources', Array( + ('error_code', Int16), + ('error_message', String('utf-8')), + ('resource_type', Int8), + ('resource_name', String('utf-8')), + ('config_entries', Array( + ('config_names', String('utf-8')), + ('config_value', String('utf-8')), + ('read_only', Boolean), + ('is_default', Boolean), + ('is_sensitive', Boolean))))) + ) + +class DescribeConfigsResponse_v1(Response): + API_KEY = 32 + API_VERSION = 1 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('resources', Array( + ('error_code', Int16), + ('error_message', String('utf-8')), + ('resource_type', Int8), + ('resource_name', String('utf-8')), + ('config_entries', Array( + ('config_names', String('utf-8')), + ('config_value', String('utf-8')), + ('read_only', Boolean), + ('is_default', Boolean), + ('is_sensitive', Boolean), + ('config_synonyms', Array( + ('config_name', String('utf-8')), + ('config_value', String('utf-8')), + ('config_source', Int8))))))) + ) + +class DescribeConfigsRequest_v0(Request): + API_KEY = 32 + API_VERSION = 0 + RESPONSE_TYPE = DescribeConfigsResponse_v0 + SCHEMA = Schema( + ('resources', Array( + ('resource_type', Int8), + ('resource_name', String('utf-8')), + ('config_names', Array(String('utf-8'))))) + ) + +class DescribeConfigsRequest_v1(Request): + API_KEY = 32 + API_VERSION = 1 + RESPONSE_TYPE = DescribeConfigsResponse_v1 + SCHEMA = Schema( + ('resources', Array( + ('resource_type', Int8), + ('resource_name', String('utf-8')), + ('config_names', Array(String('utf-8'))))), + ('include_synonyms', Boolean) + ) + +DescribeConfigsRequest = [DescribeConfigsRequest_v0, DescribeConfigsRequest_v1] +DescribeConfigsResponse = [DescribeConfigsResponse_v0, DescribeConfigsResponse_v1] + +class SaslAuthenticateResponse_v0(Request): + API_KEY = 36 + API_VERSION = 0 + SCHEMA = Schema( + ('error_code', Int16), + ('error_message', String('utf-8')), + ('sasl_auth_bytes', Bytes) + ) + + +class SaslAuthenticateRequest_v0(Request): + API_KEY = 36 + API_VERSION = 0 + RESPONSE_TYPE = SaslAuthenticateResponse_v0 + SCHEMA = Schema( + ('sasl_auth_bytes', Bytes) + ) + + +SaslAuthenticateRequest = [SaslAuthenticateRequest_v0] +SaslAuthenticateResponse = [SaslAuthenticateResponse_v0] + + +class CreatePartitionsResponse_v0(Response): + API_KEY = 37 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('topic_errors', Array( + ('topic', String('utf-8')), + ('error_code', Int16), + ('error_message', String('utf-8')))) + ) + + +class CreatePartitionsRequest_v0(Request): + API_KEY = 37 + API_VERSION = 0 + RESPONSE_TYPE = CreatePartitionsResponse_v0 + SCHEMA = Schema( + ('topic_partitions', Array( + ('topic', String('utf-8')), + ('new_partitions', Schema( + ('count', Int32), + ('assignment', Array(Array(Int32))))))), + ('timeout', Int32), + ('validate_only', Boolean) + ) + + +CreatePartitionsRequest = [CreatePartitionsRequest_v0] +CreatePartitionsResponse = [CreatePartitionsResponse_v0] diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index ec24a3993..efaf63ea2 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -2,8 +2,8 @@ import abc -from .struct import Struct -from .types import Int16, Int32, String, Schema +from kafka.protocol.struct import Struct +from kafka.protocol.types import Int16, Int32, String, Schema class RequestHeader(Struct): diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py index 9d744c782..31fc23707 100644 --- a/kafka/protocol/commit.py +++ b/kafka/protocol/commit.py @@ -1,7 +1,7 @@ from __future__ import absolute_import -from .api import Request, Response -from .types import Array, Int8, Int16, Int32, Int64, Schema, String +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String class OffsetCommitResponse_v0(Response): diff --git a/kafka/protocol/fetch.py b/kafka/protocol/fetch.py index 0b03845ee..dd3f648cf 100644 --- a/kafka/protocol/fetch.py +++ b/kafka/protocol/fetch.py @@ -1,7 +1,7 @@ from __future__ import absolute_import -from .api import Request, Response -from .types import Array, Int8, Int16, Int32, Int64, Schema, String, Bytes +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String, Bytes class FetchResponse_v0(Response): @@ -84,6 +84,16 @@ class FetchResponse_v5(Response): ) +class FetchResponse_v6(Response): + """ + Same as FetchResponse_v5. The version number is bumped up to indicate that the client supports KafkaStorageException. + The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5 + """ + API_KEY = 1 + API_VERSION = 6 + SCHEMA = FetchResponse_v5.SCHEMA + + class FetchRequest_v0(Request): API_KEY = 1 API_VERSION = 0 @@ -174,11 +184,25 @@ class FetchRequest_v5(Request): ) +class FetchRequest_v6(Request): + """ + The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5. + The version number is bumped up to indicate that the client supports KafkaStorageException. + The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5 + """ + API_KEY = 1 + API_VERSION = 6 + RESPONSE_TYPE = FetchResponse_v6 + SCHEMA = FetchRequest_v5.SCHEMA + + FetchRequest = [ FetchRequest_v0, FetchRequest_v1, FetchRequest_v2, - FetchRequest_v3, FetchRequest_v4, FetchRequest_v5 + FetchRequest_v3, FetchRequest_v4, FetchRequest_v5, + FetchRequest_v6 ] FetchResponse = [ FetchResponse_v0, FetchResponse_v1, FetchResponse_v2, - FetchResponse_v3, FetchResponse_v4, FetchResponse_v5 + FetchResponse_v3, FetchResponse_v4, FetchResponse_v5, + FetchResponse_v6 ] diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index c6acca83f..bcb96553b 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -1,8 +1,8 @@ from __future__ import absolute_import -from .api import Request, Response -from .struct import Struct -from .types import Array, Bytes, Int16, Int32, Schema, String +from kafka.protocol.api import Request, Response +from kafka.protocol.struct import Struct +from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String class JoinGroupResponse_v0(Response): @@ -87,7 +87,7 @@ class JoinGroupRequest_v2(Request): JoinGroupRequest_v0, JoinGroupRequest_v1, JoinGroupRequest_v2 ] JoinGroupResponse = [ - JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v1 + JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v2 ] diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py index b8f84e717..7dd258032 100644 --- a/kafka/protocol/legacy.py +++ b/kafka/protocol/legacy.py @@ -15,7 +15,6 @@ from kafka.codec import gzip_encode, snappy_encode from kafka.errors import ProtocolError, UnsupportedCodecError -from kafka.structs import ConsumerMetadataResponse from kafka.util import ( crc32, read_short_string, relative_unpack, write_int_string, group_by_topic_and_partition) @@ -322,7 +321,7 @@ def encode_consumer_metadata_request(cls, client_id, correlation_id, payloads): @classmethod def decode_consumer_metadata_response(cls, data): """ - Decode bytes to a ConsumerMetadataResponse + Decode bytes to a kafka.structs.ConsumerMetadataResponse Arguments: data: bytes to decode @@ -331,7 +330,7 @@ def decode_consumer_metadata_response(cls, data): (host, cur) = read_short_string(data, cur) ((port,), cur) = relative_unpack('>i', data, cur) - return ConsumerMetadataResponse(error, nodeId, host, port) + return kafka.structs.ConsumerMetadataResponse(error, nodeId, host, port) @classmethod def encode_offset_commit_request(cls, group, payloads): diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index a330ed805..19dcbd9de 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -3,15 +3,15 @@ import io import time -from ..codec import (has_gzip, has_snappy, has_lz4, +from kafka.codec import (has_gzip, has_snappy, has_lz4, gzip_decode, snappy_decode, lz4_decode, lz4_decode_old_kafka) -from .frame import KafkaBytes -from .struct import Struct -from .types import ( +from kafka.protocol.frame import KafkaBytes +from kafka.protocol.struct import Struct +from kafka.protocol.types import ( Int8, Int32, Int64, Bytes, Schema, AbstractType ) -from ..util import crc32, WeakMethod +from kafka.util import crc32, WeakMethod class Message(Struct): diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py index 2be82090e..414e5b84a 100644 --- a/kafka/protocol/metadata.py +++ b/kafka/protocol/metadata.py @@ -1,7 +1,7 @@ from __future__ import absolute_import -from .api import Request, Response -from .types import Array, Boolean, Int16, Int32, Schema, String +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Boolean, Int16, Int32, Schema, String class MetadataResponse_v0(Response): @@ -102,6 +102,32 @@ class MetadataResponse_v4(Response): SCHEMA = MetadataResponse_v3.SCHEMA +class MetadataResponse_v5(Response): + API_KEY = 3 + API_VERSION = 5 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('brokers', Array( + ('node_id', Int32), + ('host', String('utf-8')), + ('port', Int32), + ('rack', String('utf-8')))), + ('cluster_id', String('utf-8')), + ('controller_id', Int32), + ('topics', Array( + ('error_code', Int16), + ('topic', String('utf-8')), + ('is_internal', Boolean), + ('partitions', Array( + ('error_code', Int16), + ('partition', Int32), + ('leader', Int32), + ('replicas', Array(Int32)), + ('isr', Array(Int32)), + ('offline_replicas', Array(Int32)))))) + ) + + class MetadataRequest_v0(Request): API_KEY = 3 API_VERSION = 0 @@ -151,11 +177,24 @@ class MetadataRequest_v4(Request): NO_TOPICS = None # Empty array (len 0) for topics returns no topics +class MetadataRequest_v5(Request): + """ + The v5 metadata request is the same as v4. + An additional field for offline_replicas has been added to the v5 metadata response + """ + API_KEY = 3 + API_VERSION = 5 + RESPONSE_TYPE = MetadataResponse_v5 + SCHEMA = MetadataRequest_v4.SCHEMA + ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics + NO_TOPICS = None # Empty array (len 0) for topics returns no topics + + MetadataRequest = [ MetadataRequest_v0, MetadataRequest_v1, MetadataRequest_v2, - MetadataRequest_v3, MetadataRequest_v4 + MetadataRequest_v3, MetadataRequest_v4, MetadataRequest_v5 ] MetadataResponse = [ MetadataResponse_v0, MetadataResponse_v1, MetadataResponse_v2, - MetadataResponse_v3, MetadataResponse_v4 + MetadataResponse_v3, MetadataResponse_v4, MetadataResponse_v5 ] diff --git a/kafka/protocol/offset.py b/kafka/protocol/offset.py index 517965836..3c254de40 100644 --- a/kafka/protocol/offset.py +++ b/kafka/protocol/offset.py @@ -1,7 +1,7 @@ from __future__ import absolute_import -from .api import Request, Response -from .types import Array, Int8, Int16, Int32, Int64, Schema, String +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String UNKNOWN_OFFSET = -1 diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py index 34ff949ef..f4032b311 100644 --- a/kafka/protocol/produce.py +++ b/kafka/protocol/produce.py @@ -1,7 +1,7 @@ from __future__ import absolute_import -from .api import Request, Response -from .types import Int16, Int32, Int64, String, Array, Schema, Bytes +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Int16, Int32, Int64, String, Array, Schema, Bytes class ProduceResponse_v0(Response): @@ -52,52 +52,67 @@ class ProduceResponse_v3(Response): SCHEMA = ProduceResponse_v2.SCHEMA -class ProduceRequest_v0(Request): +class ProduceResponse_v4(Response): + """ + The version number is bumped up to indicate that the client supports KafkaStorageException. + The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3 + """ API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = ProduceResponse_v0 + API_VERSION = 4 + SCHEMA = ProduceResponse_v3.SCHEMA + + +class ProduceResponse_v5(Response): + API_KEY = 0 + API_VERSION = 5 SCHEMA = Schema( - ('required_acks', Int16), - ('timeout', Int32), ('topics', Array( ('topic', String('utf-8')), ('partitions', Array( ('partition', Int32), - ('messages', Bytes))))) + ('error_code', Int16), + ('offset', Int64), + ('timestamp', Int64), + ('log_start_offset', Int64))))), + ('throttle_time_ms', Int32) ) + +class ProduceRequest(Request): + API_KEY = 0 + def expect_response(self): if self.required_acks == 0: # pylint: disable=no-member return False return True -class ProduceRequest_v1(Request): - API_KEY = 0 +class ProduceRequest_v0(ProduceRequest): + API_VERSION = 0 + RESPONSE_TYPE = ProduceResponse_v0 + SCHEMA = Schema( + ('required_acks', Int16), + ('timeout', Int32), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('messages', Bytes))))) + ) + + +class ProduceRequest_v1(ProduceRequest): API_VERSION = 1 RESPONSE_TYPE = ProduceResponse_v1 SCHEMA = ProduceRequest_v0.SCHEMA - def expect_response(self): - if self.required_acks == 0: # pylint: disable=no-member - return False - return True - - -class ProduceRequest_v2(Request): - API_KEY = 0 +class ProduceRequest_v2(ProduceRequest): API_VERSION = 2 RESPONSE_TYPE = ProduceResponse_v2 SCHEMA = ProduceRequest_v1.SCHEMA - def expect_response(self): - if self.required_acks == 0: # pylint: disable=no-member - return False - return True - -class ProduceRequest_v3(Request): - API_KEY = 0 +class ProduceRequest_v3(ProduceRequest): API_VERSION = 3 RESPONSE_TYPE = ProduceResponse_v3 SCHEMA = Schema( @@ -111,17 +126,32 @@ class ProduceRequest_v3(Request): ('messages', Bytes))))) ) - def expect_response(self): - if self.required_acks == 0: # pylint: disable=no-member - return False - return True + +class ProduceRequest_v4(ProduceRequest): + """ + The version number is bumped up to indicate that the client supports KafkaStorageException. + The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3 + """ + API_VERSION = 4 + RESPONSE_TYPE = ProduceResponse_v4 + SCHEMA = ProduceRequest_v3.SCHEMA + + +class ProduceRequest_v5(ProduceRequest): + """ + Same as v4. The version number is bumped since the v5 response includes an additional + partition level field: the log_start_offset. + """ + API_VERSION = 5 + RESPONSE_TYPE = ProduceResponse_v5 + SCHEMA = ProduceRequest_v4.SCHEMA ProduceRequest = [ ProduceRequest_v0, ProduceRequest_v1, ProduceRequest_v2, - ProduceRequest_v3 + ProduceRequest_v3, ProduceRequest_v4, ProduceRequest_v5 ] ProduceResponse = [ ProduceResponse_v0, ProduceResponse_v1, ProduceResponse_v2, - ProduceResponse_v2 + ProduceResponse_v3, ProduceResponse_v4, ProduceResponse_v5 ] diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index 3288172cf..676de1ba4 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -2,10 +2,10 @@ from io import BytesIO -from .abstract import AbstractType -from .types import Schema +from kafka.protocol.abstract import AbstractType +from kafka.protocol.types import Schema -from ..util import WeakMethod +from kafka.util import WeakMethod class Struct(AbstractType): diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 6a6e89e41..5ccb83ea7 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -2,7 +2,7 @@ from struct import pack, unpack, error -from .abstract import AbstractType +from kafka.protocol.abstract import AbstractType def _pack(f, value): @@ -10,7 +10,7 @@ def _pack(f, value): return pack(f, value) except error as e: raise ValueError("Error encountered when attempting to convert value: " - "{} to struct format: '{}', hit error: {}" + "{!r} to struct format: '{}', hit error: {}" .format(value, f, e)) @@ -20,8 +20,8 @@ def _unpack(f, data): return value except error as e: raise ValueError("Error encountered when attempting to convert value: " - "{} to struct format: '{}', hit error: {}" - .format(value, f, e)) + "{!r} to struct format: '{}', hit error: {}" + .format(data, f, e)) class Int8(AbstractType): diff --git a/kafka/record/__init__.py b/kafka/record/__init__.py index 4c75acb13..93936df48 100644 --- a/kafka/record/__init__.py +++ b/kafka/record/__init__.py @@ -1,3 +1,3 @@ -from .memory_records import MemoryRecords +from kafka.record.memory_records import MemoryRecords, MemoryRecordsBuilder -__all__ = ["MemoryRecords"] +__all__ = ["MemoryRecords", "MemoryRecordsBuilder"] diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index 3d517af25..955e3ee2a 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -54,17 +54,18 @@ # * Timestamp Type (3) # * Compression Type (0-2) -import io import struct import time -from .abc import ABCRecord, ABCRecordBatch, ABCRecordBatchBuilder -from .util import decode_varint, encode_varint, calc_crc32c, size_of_varint - -from kafka.errors import CorruptRecordException +from kafka.record.abc import ABCRecord, ABCRecordBatch, ABCRecordBatchBuilder +from kafka.record.util import ( + decode_varint, encode_varint, calc_crc32c, size_of_varint +) +from kafka.errors import CorruptRecordException, UnsupportedCodecError from kafka.codec import ( gzip_encode, snappy_encode, lz4_encode, gzip_decode, snappy_decode, lz4_decode ) +import kafka.codec as codecs class DefaultRecordBase(object): @@ -101,6 +102,17 @@ class DefaultRecordBase(object): LOG_APPEND_TIME = 1 CREATE_TIME = 0 + def _assert_has_codec(self, compression_type): + if compression_type == self.CODEC_GZIP: + checker, name = codecs.has_gzip, "gzip" + elif compression_type == self.CODEC_SNAPPY: + checker, name = codecs.has_snappy, "snappy" + elif compression_type == self.CODEC_LZ4: + checker, name = codecs.has_lz4, "lz4" + if not checker(): + raise UnsupportedCodecError( + "Libraries for {} compression codec not found".format(name)) + class DefaultRecordBatch(DefaultRecordBase, ABCRecordBatch): @@ -156,6 +168,7 @@ def _maybe_uncompress(self): if not self._decompressed: compression_type = self.compression_type if compression_type != self.CODEC_NONE: + self._assert_has_codec(compression_type) data = memoryview(self._buffer)[self._pos:] if compression_type == self.CODEC_GZIP: uncompressed = gzip_decode(data) @@ -237,7 +250,7 @@ def _read_msg( # validate whether we have read all header bytes in the current record if pos - start_pos != length: - CorruptRecordException( + raise CorruptRecordException( "Invalid record size: expected to read {} bytes in record " "payload, but instead read {}".format(length, pos - start_pos)) self._pos = pos @@ -481,6 +494,7 @@ def write_header(self, use_compression_type=True): def _maybe_compress(self): if self._compression_type != self.CODEC_NONE: + self._assert_has_codec(self._compression_type) header_size = self.HEADER_STRUCT.size data = bytes(self._buffer[header_size:]) if self._compression_type == self.CODEC_GZIP: diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index 8c0791ef9..1bdba8152 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -44,14 +44,15 @@ import struct import time -from .abc import ABCRecord, ABCRecordBatch, ABCRecordBatchBuilder -from .util import calc_crc32 +from kafka.record.abc import ABCRecord, ABCRecordBatch, ABCRecordBatchBuilder +from kafka.record.util import calc_crc32 from kafka.codec import ( gzip_encode, snappy_encode, lz4_encode, lz4_encode_old_kafka, - gzip_decode, snappy_decode, lz4_decode, lz4_decode_old_kafka + gzip_decode, snappy_decode, lz4_decode, lz4_decode_old_kafka, ) -from kafka.errors import CorruptRecordException +import kafka.codec as codecs +from kafka.errors import CorruptRecordException, UnsupportedCodecError class LegacyRecordBase(object): @@ -112,6 +113,17 @@ class LegacyRecordBase(object): NO_TIMESTAMP = -1 + def _assert_has_codec(self, compression_type): + if compression_type == self.CODEC_GZIP: + checker, name = codecs.has_gzip, "gzip" + elif compression_type == self.CODEC_SNAPPY: + checker, name = codecs.has_snappy, "snappy" + elif compression_type == self.CODEC_LZ4: + checker, name = codecs.has_lz4, "lz4" + if not checker(): + raise UnsupportedCodecError( + "Libraries for {} compression codec not found".format(name)) + class LegacyRecordBatch(ABCRecordBatch, LegacyRecordBase): @@ -166,6 +178,7 @@ def _decompress(self, key_offset): data = self._buffer[pos:pos + value_size] compression_type = self.compression_type + self._assert_has_codec(compression_type) if compression_type == self.CODEC_GZIP: uncompressed = gzip_decode(data) elif compression_type == self.CODEC_SNAPPY: @@ -419,6 +432,7 @@ def _encode_msg(self, start_pos, offset, timestamp, key, value, def _maybe_compress(self): if self._compression_type: + self._assert_has_codec(self._compression_type) data = bytes(self._buffer) if self._compression_type == self.CODEC_GZIP: compressed = gzip_encode(data) diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index 56aa51faa..f67c4fe3a 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -18,13 +18,14 @@ # # So we can iterate over batches just by knowing offsets of Length. Magic is # used to construct the correct class for Batch itself. +from __future__ import division import struct from kafka.errors import CorruptRecordException -from .abc import ABCRecords -from .legacy_records import LegacyRecordBatch, LegacyRecordBatchBuilder -from .default_records import DefaultRecordBatch, DefaultRecordBatchBuilder +from kafka.record.abc import ABCRecords +from kafka.record.legacy_records import LegacyRecordBatch, LegacyRecordBatchBuilder +from kafka.record.default_records import DefaultRecordBatch, DefaultRecordBatchBuilder class MemoryRecords(ABCRecords): @@ -131,15 +132,14 @@ def __init__(self, magic, compression_type, batch_size): def append(self, timestamp, key, value, headers=[]): """ Append a message to the buffer. - Returns: - (int, int): checksum and bytes written + Returns: RecordMetadata or None if unable to append """ if self._closed: - return None, 0 + return None offset = self._next_offset metadata = self._builder.append(offset, timestamp, key, value, headers) - # Return of 0 size means there's no space to add a new message + # Return of None means there's no space to add a new message if metadata is None: return None diff --git a/kafka/record/util.py b/kafka/record/util.py index 88135f1a7..74b9a69b0 100644 --- a/kafka/record/util.py +++ b/kafka/record/util.py @@ -1,6 +1,10 @@ import binascii -from ._crc32c import crc as crc32c_py +from kafka.record._crc32c import crc as crc32c_py +try: + from crc32c import crc32 as crc32c_c +except ImportError: + crc32c_c = None def encode_varint(value, write): @@ -113,11 +117,15 @@ def decode_varint(buffer, pos=0): raise ValueError("Out of int64 range") -def calc_crc32c(memview): +_crc32c = crc32c_py +if crc32c_c is not None: + _crc32c = crc32c_c + + +def calc_crc32c(memview, _crc32c=_crc32c): """ Calculate CRC-32C (Castagnoli) checksum over a memoryview of data """ - crc = crc32c_py(memview) - return crc + return _crc32c(memview) def calc_crc32(memview): diff --git a/kafka/serializer/__init__.py b/kafka/serializer/__init__.py index c08cffe89..90cd93ab2 100644 --- a/kafka/serializer/__init__.py +++ b/kafka/serializer/__init__.py @@ -1,3 +1,3 @@ from __future__ import absolute_import -from .abstract import Serializer, Deserializer +from kafka.serializer.abstract import Serializer, Deserializer diff --git a/kafka/structs.py b/kafka/structs.py index 62f36dd4c..e15e92ed6 100644 --- a/kafka/structs.py +++ b/kafka/structs.py @@ -93,7 +93,3 @@ # Limit value: int >= 0, 0 means no retries RetryOptions = namedtuple("RetryOptions", ["limit", "backoff_ms", "retry_on_timeouts"]) - - -# Support legacy imports from kafka.common -from kafka.errors import * diff --git a/kafka/version.py b/kafka/version.py index 7ca88b023..5f686fe0f 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '1.3.6.dev' +__version__ = '1.4.4.dev' diff --git a/pylint.rc b/pylint.rc index d13ef519e..d22e523ec 100644 --- a/pylint.rc +++ b/pylint.rc @@ -1,5 +1,6 @@ [TYPECHECK] ignored-classes=SyncManager,_socketobject +generated-members=py.* [MESSAGES CONTROL] disable=E1129 diff --git a/requirements-dev.txt b/requirements-dev.txt index 249eb23f4..b98b58ab9 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -1,16 +1,15 @@ flake8==3.4.1 -pytest==3.2.2 +pytest==3.4.0 pytest-cov==2.5.1 -pytest-catchlog==1.2.2 docker-py==1.10.6 coveralls==1.2.0 Sphinx==1.6.4 -lz4==0.11.1 +lz4==0.19.1 xxhash==1.0.1 python-snappy==0.5.1 tox==2.9.1 -pylint==1.8.0 +pylint==1.8.2 pytest-pylint==0.7.1 -# pytest-sugar==0.9.0 pytest-mock==1.6.3 sphinx-rtd-theme==0.2.4 +crc32c==1.2 diff --git a/servers/1.0.1/resources/kafka.properties b/servers/1.0.1/resources/kafka.properties new file mode 100644 index 000000000..f08855ce6 --- /dev/null +++ b/servers/1.0.1/resources/kafka.properties @@ -0,0 +1,142 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id={broker_id} + +############################# Socket Server Settings ############################# + +listeners={transport}://{host}:{port} +security.inter.broker.protocol={transport} + +ssl.keystore.location={ssl_dir}/server.keystore.jks +ssl.keystore.password=foobar +ssl.key.password=foobar +ssl.truststore.location={ssl_dir}/server.truststore.jks +ssl.truststore.password=foobar + +# The port the socket server listens on +#port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=3 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=102400 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs={tmp_dir}/data + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions={partitions} +default.replication.factor={replicas} + +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# 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=1 + +# Allow shorter session timeouts for tests +group.min.session.timeout.ms=1000 + + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=30000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/1.0.1/resources/log4j.properties b/servers/1.0.1/resources/log4j.properties new file mode 100644 index 000000000..b0b76aa79 --- /dev/null +++ b/servers/1.0.1/resources/log4j.properties @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=INFO, stdout, logfile + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.logfile=org.apache.log4j.FileAppender +log4j.appender.logfile.File=${kafka.logs.dir}/server.log +log4j.appender.logfile.layout=org.apache.log4j.PatternLayout +log4j.appender.logfile.layout.ConversionPattern=[%d] %p %m (%c)%n diff --git a/servers/1.0.1/resources/zookeeper.properties b/servers/1.0.1/resources/zookeeper.properties new file mode 100644 index 000000000..e3fd09742 --- /dev/null +++ b/servers/1.0.1/resources/zookeeper.properties @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir={tmp_dir} +# the port at which the clients will connect +clientPort={port} +clientPortAddress={host} +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 diff --git a/test/conftest.py b/test/conftest.py index e85b977c8..dbc2378d9 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -1,38 +1,117 @@ from __future__ import absolute_import -import os +import inspect import pytest from test.fixtures import KafkaFixture, ZookeeperFixture - +from test.testutil import kafka_version, 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('.'))) - + """Return the Kafka version set in the OS environment""" + return kafka_version() @pytest.fixture(scope="module") -def zookeeper(version, request): - assert version - zk = ZookeeperFixture.instance() - yield zk - zk.close() +def zookeeper(): + """Return a Zookeeper fixture""" + zk_instance = ZookeeperFixture.instance() + yield zk_instance + zk_instance.close() +@pytest.fixture(scope="module") +def kafka_broker(kafka_broker_factory): + """Return a Kafka broker fixture""" + return kafka_broker_factory()[0] @pytest.fixture(scope="module") -def kafka_broker(version, zookeeper, request): - assert version - k = KafkaFixture.instance(0, zookeeper.host, zookeeper.port, - partitions=4) - yield k - k.close() +def kafka_broker_factory(version, zookeeper): + """Return a Kafka broker fixture factory""" + assert version, 'KAFKA_VERSION must be specified to run integration tests' + + _brokers = [] + def factory(**broker_params): + params = {} if broker_params is None else broker_params.copy() + params.setdefault('partitions', 4) + num_brokers = params.pop('num_brokers', 1) + brokers = tuple(KafkaFixture.instance(x, zookeeper, **params) + for x in range(num_brokers)) + _brokers.extend(brokers) + return brokers + yield factory + + for broker in _brokers: + broker.close() + +@pytest.fixture +def simple_client(kafka_broker, request, topic): + """Return a SimpleClient fixture""" + client = kafka_broker.get_simple_client(client_id='%s_client' % (request.node.name,)) + client.ensure_topic_exists(topic) + yield client + client.close() + +@pytest.fixture +def kafka_client(kafka_broker, request): + """Return a KafkaClient fixture""" + (client,) = kafka_broker.get_clients(cnt=1, client_id='%s_client' % (request.node.name,)) + yield client + client.close() + +@pytest.fixture +def kafka_consumer(kafka_consumer_factory): + """Return a KafkaConsumer fixture""" + return kafka_consumer_factory() + +@pytest.fixture +def kafka_consumer_factory(kafka_broker, topic, request): + """Return a KafkaConsumer factory fixture""" + _consumer = [None] + + def factory(**kafka_consumer_params): + params = {} if kafka_consumer_params is None else kafka_consumer_params.copy() + params.setdefault('client_id', 'consumer_%s' % (request.node.name,)) + params.setdefault('auto_offset_reset', 'earliest') + _consumer[0] = next(kafka_broker.get_consumers(cnt=1, topics=[topic], **params)) + return _consumer[0] + + yield factory + + if _consumer[0]: + _consumer[0].close() + +@pytest.fixture +def kafka_producer(kafka_producer_factory): + """Return a KafkaProducer fixture""" + yield kafka_producer_factory() + +@pytest.fixture +def kafka_producer_factory(kafka_broker, request): + """Return a KafkaProduce factory fixture""" + _producer = [None] + + def factory(**kafka_producer_params): + params = {} if kafka_producer_params is None else kafka_producer_params.copy() + params.setdefault('client_id', 'producer_%s' % (request.node.name,)) + _producer[0] = next(kafka_broker.get_producers(cnt=1, **params)) + return _producer[0] + + yield factory + + if _producer[0]: + _producer[0].close() + +@pytest.fixture +def topic(kafka_broker, request): + """Return a topic fixture""" + topic_name = '%s_%s' % (request.node.name, random_string(10)) + kafka_broker.create_topics([topic_name]) + return topic_name @pytest.fixture def conn(mocker): + """Return a connection mocker fixture""" from kafka.conn import ConnectionStates from kafka.future import Future from kafka.protocol.metadata import MetadataResponse @@ -49,6 +128,7 @@ def _set_conn_state(state): return state conn._set_conn_state = _set_conn_state conn.connect.side_effect = lambda: conn.state + conn.connect_blocking.return_value = True conn.connecting = lambda: conn.state in (ConnectionStates.CONNECTING, ConnectionStates.HANDSHAKE) conn.connected = lambda: conn.state is ConnectionStates.CONNECTED diff --git a/test/fixtures.py b/test/fixtures.py index 62c6d50dd..493a664a5 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -4,29 +4,55 @@ import logging import os import os.path -import shutil +import random +import socket +import string import subprocess -import tempfile import time import uuid -from six.moves import urllib +import py +from six.moves import urllib, xrange from six.moves.urllib.parse import urlparse # pylint: disable=E0611,F0401 +from kafka import errors, KafkaConsumer, KafkaProducer, SimpleClient +from kafka.client_async import KafkaClient +from kafka.protocol.admin import CreateTopicsRequest +from kafka.protocol.metadata import MetadataRequest from test.service import ExternalService, SpawnedService -from test.testutil import get_open_port - log = logging.getLogger(__name__) +def random_string(length): + return "".join(random.choice(string.ascii_letters) for i in xrange(length)) + +def version_str_to_list(version_str): + return tuple(map(int, version_str.split('.'))) # e.g., (0, 8, 1, 1) + +def version(): + if 'KAFKA_VERSION' not in os.environ: + return () + return version_str_to_list(os.environ['KAFKA_VERSION']) + +def get_open_port(): + sock = socket.socket() + sock.bind(("", 0)) + port = sock.getsockname()[1] + sock.close() + return port class Fixture(object): kafka_version = os.environ.get('KAFKA_VERSION', '0.11.0.2') scala_version = os.environ.get("SCALA_VERSION", '2.8.0') - project_root = os.environ.get('PROJECT_ROOT', os.path.abspath(os.path.join(os.path.dirname(__file__), ".."))) - kafka_root = os.environ.get("KAFKA_ROOT", os.path.join(project_root, 'servers', kafka_version, "kafka-bin")) + project_root = os.environ.get('PROJECT_ROOT', + os.path.abspath(os.path.join(os.path.dirname(__file__), ".."))) + kafka_root = os.environ.get("KAFKA_ROOT", + os.path.join(project_root, 'servers', kafka_version, "kafka-bin")) ivy_root = os.environ.get('IVY_ROOT', os.path.expanduser("~/.ivy2/cache")) + def __init__(self): + self.child = None + @classmethod def download_official_distribution(cls, kafka_version=None, @@ -71,31 +97,34 @@ def test_resource(cls, filename): @classmethod def kafka_run_class_args(cls, *args): result = [os.path.join(cls.kafka_root, 'bin', 'kafka-run-class.sh')] - result.extend(args) + result.extend([str(arg) for arg in args]) return result def kafka_run_class_env(self): env = os.environ.copy() - env['KAFKA_LOG4J_OPTS'] = "-Dlog4j.configuration=file:%s" % self.test_resource("log4j.properties") + env['KAFKA_LOG4J_OPTS'] = "-Dlog4j.configuration=file:%s" % \ + self.test_resource("log4j.properties") return env @classmethod def render_template(cls, source_file, target_file, binding): - log.info('Rendering %s from template %s', target_file, source_file) + log.info('Rendering %s from template %s', target_file.strpath, source_file) with open(source_file, "r") as handle: template = handle.read() assert len(template) > 0, 'Empty template %s' % source_file - with open(target_file, "w") as handle: + with open(target_file.strpath, "w") as handle: handle.write(template.format(**binding)) handle.flush() os.fsync(handle) # fsync directory for durability # https://blog.gocept.com/2013/07/15/reliable-file-updates-with-python/ - dirfd = os.open(os.path.dirname(target_file), os.O_DIRECTORY) + dirfd = os.open(os.path.dirname(target_file.strpath), os.O_DIRECTORY) os.fsync(dirfd) os.close(dirfd) + def dump_logs(self): + self.child.dump_logs() class ZookeeperFixture(Fixture): @classmethod @@ -111,37 +140,41 @@ def instance(cls): fixture.open() return fixture - def __init__(self, host, port): + def __init__(self, host, port, tmp_dir=None): + super(ZookeeperFixture, self).__init__() self.host = host self.port = port - self.tmp_dir = None - self.child = None + self.tmp_dir = tmp_dir def kafka_run_class_env(self): env = super(ZookeeperFixture, self).kafka_run_class_env() - env['LOG_DIR'] = os.path.join(self.tmp_dir, 'logs') + env['LOG_DIR'] = self.tmp_dir.join('logs').strpath return env def out(self, message): log.info("*** Zookeeper [%s:%s]: %s", self.host, self.port or '(auto)', message) def open(self): - self.tmp_dir = tempfile.mkdtemp() + if self.tmp_dir is None: + self.tmp_dir = py.path.local.mkdtemp() #pylint: disable=no-member + self.tmp_dir.ensure(dir=True) + self.out("Running local instance...") log.info(" host = %s", self.host) log.info(" port = %s", self.port or '(auto)') - log.info(" tmp_dir = %s", self.tmp_dir) + log.info(" tmp_dir = %s", self.tmp_dir.strpath) # Configure Zookeeper child process template = self.test_resource("zookeeper.properties") - properties = os.path.join(self.tmp_dir, "zookeeper.properties") - args = self.kafka_run_class_args("org.apache.zookeeper.server.quorum.QuorumPeerMain", properties) + properties = self.tmp_dir.join("zookeeper.properties") + args = self.kafka_run_class_args("org.apache.zookeeper.server.quorum.QuorumPeerMain", + properties.strpath) env = self.kafka_run_class_env() # Party! timeout = 5 - max_timeout = 30 + max_timeout = 120 backoff = 1 end_at = time.time() + max_timeout tries = 1 @@ -161,6 +194,7 @@ def open(self): timeout *= 2 time.sleep(backoff) tries += 1 + backoff += 1 else: raise RuntimeError('Failed to start Zookeeper before max_timeout') self.out("Done!") @@ -173,7 +207,7 @@ def close(self): self.child.stop() self.child = None self.out("Done!") - shutil.rmtree(self.tmp_dir) + self.tmp_dir.remove() def __del__(self): self.close() @@ -181,9 +215,11 @@ def __del__(self): class KafkaFixture(Fixture): @classmethod - def instance(cls, broker_id, zk_host, zk_port, zk_chroot=None, + def instance(cls, broker_id, zookeeper, zk_chroot=None, host=None, port=None, - transport='PLAINTEXT', replicas=1, partitions=2): + transport='PLAINTEXT', replicas=1, partitions=2, + sasl_mechanism='PLAIN', auto_create_topic=True, tmp_dir=None): + if zk_chroot is None: zk_chroot = "kafka-python_" + str(uuid.uuid4()).replace("-", "_") if "KAFKA_URI" in os.environ: @@ -194,19 +230,29 @@ def instance(cls, broker_id, zk_host, zk_port, zk_chroot=None, if host is None: host = "localhost" fixture = KafkaFixture(host, port, broker_id, - zk_host, zk_port, zk_chroot, + zookeeper, zk_chroot, transport=transport, - replicas=replicas, partitions=partitions) + replicas=replicas, partitions=partitions, + sasl_mechanism=sasl_mechanism, + auto_create_topic=auto_create_topic, + tmp_dir=tmp_dir) + fixture.open() return fixture - def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot, - replicas=1, partitions=2, transport='PLAINTEXT'): + def __init__(self, host, port, broker_id, zookeeper, zk_chroot, + replicas=1, partitions=2, transport='PLAINTEXT', + sasl_mechanism='PLAIN', auto_create_topic=True, + tmp_dir=None): + super(KafkaFixture, self).__init__() + self.host = host self.port = port self.broker_id = broker_id + self.auto_create_topic = auto_create_topic self.transport = transport.upper() + self.sasl_mechanism = sasl_mechanism.upper() self.ssl_dir = self.test_resource('ssl') # TODO: checking for port connection would be better than scanning logs @@ -214,71 +260,59 @@ def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot, # The logging format changed slightly in 1.0.0 self.start_pattern = r"\[Kafka ?Server (id=)?%d\],? started" % broker_id - self.zk_host = zk_host - self.zk_port = zk_port + self.zookeeper = zookeeper self.zk_chroot = zk_chroot + # Add the attributes below for the template binding + self.zk_host = self.zookeeper.host + self.zk_port = self.zookeeper.port self.replicas = replicas self.partitions = partitions - self.tmp_dir = None - self.child = None + self.tmp_dir = tmp_dir self.running = False + self._client = None + + def bootstrap_server(self): + return '%s:%d' % (self.host, self.port) + def kafka_run_class_env(self): env = super(KafkaFixture, self).kafka_run_class_env() - env['LOG_DIR'] = os.path.join(self.tmp_dir, 'logs') + env['LOG_DIR'] = self.tmp_dir.join('logs').strpath return env def out(self, message): log.info("*** Kafka [%s:%s]: %s", self.host, self.port or '(auto)', message) - def open(self): - if self.running: - self.out("Instance already running") - return - - self.tmp_dir = tempfile.mkdtemp() - self.out("Running local instance...") - log.info(" host = %s", self.host) - log.info(" port = %s", self.port or '(auto)') - log.info(" transport = %s", self.transport) - log.info(" broker_id = %s", self.broker_id) - log.info(" zk_host = %s", self.zk_host) - log.info(" zk_port = %s", self.zk_port) - log.info(" zk_chroot = %s", self.zk_chroot) - log.info(" replicas = %s", self.replicas) - log.info(" partitions = %s", self.partitions) - log.info(" tmp_dir = %s", self.tmp_dir) - - # Create directories - os.mkdir(os.path.join(self.tmp_dir, "logs")) - os.mkdir(os.path.join(self.tmp_dir, "data")) - + def _create_zk_chroot(self): self.out("Creating Zookeeper chroot node...") args = self.kafka_run_class_args("org.apache.zookeeper.ZooKeeperMain", - "-server", "%s:%d" % (self.zk_host, self.zk_port), + "-server", + "%s:%d" % (self.zookeeper.host, + self.zookeeper.port), "create", "/%s" % self.zk_chroot, "kafka-python") env = self.kafka_run_class_env() proc = subprocess.Popen(args, env=env, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - if proc.wait() != 0: + if proc.wait() != 0 or proc.returncode != 0: self.out("Failed to create Zookeeper chroot node") self.out(proc.stdout.read()) self.out(proc.stderr.read()) raise RuntimeError("Failed to create Zookeeper chroot node") - self.out("Done!") + self.out("Kafka chroot created in Zookeeper!") + def start(self): # Configure Kafka child process - properties = os.path.join(self.tmp_dir, "kafka.properties") + properties = self.tmp_dir.join("kafka.properties") template = self.test_resource("kafka.properties") - args = self.kafka_run_class_args("kafka.Kafka", properties) + args = self.kafka_run_class_args("kafka.Kafka", properties.strpath) env = self.kafka_run_class_env() timeout = 5 - max_timeout = 30 + max_timeout = 120 backoff = 1 end_at = time.time() + max_timeout tries = 1 @@ -301,16 +335,48 @@ def open(self): timeout *= 2 time.sleep(backoff) tries += 1 + backoff += 1 else: raise RuntimeError('Failed to start KafkaInstance before max_timeout') + + (self._client,) = self.get_clients(1, '_internal_client') + self.out("Done!") self.running = True + + def open(self): + if self.running: + self.out("Instance already running") + return + + # Create directories + if self.tmp_dir is None: + self.tmp_dir = py.path.local.mkdtemp() #pylint: disable=no-member + self.tmp_dir.ensure(dir=True) + self.tmp_dir.ensure('logs', dir=True) + self.tmp_dir.ensure('data', dir=True) + + self.out("Running local instance...") + log.info(" host = %s", self.host) + log.info(" port = %s", self.port or '(auto)') + log.info(" transport = %s", self.transport) + log.info(" broker_id = %s", self.broker_id) + log.info(" zk_host = %s", self.zookeeper.host) + log.info(" zk_port = %s", self.zookeeper.port) + log.info(" zk_chroot = %s", self.zk_chroot) + log.info(" replicas = %s", self.replicas) + log.info(" partitions = %s", self.partitions) + log.info(" tmp_dir = %s", self.tmp_dir.strpath) + + self._create_zk_chroot() + self.start() + atexit.register(self.close) def __del__(self): self.close() - def close(self): + def stop(self): if not self.running: self.out("Instance already stopped") return @@ -318,6 +384,117 @@ def close(self): self.out("Stopping...") self.child.stop() self.child = None - self.out("Done!") - shutil.rmtree(self.tmp_dir) self.running = False + self.out("Stopped!") + + def close(self): + self.stop() + if self.tmp_dir is not None: + self.tmp_dir.remove() + self.tmp_dir = None + self.out("Done!") + + def dump_logs(self): + super(KafkaFixture, self).dump_logs() + self.zookeeper.dump_logs() + + def _send_request(self, request, timeout=None): + def _failure(error): + raise error + retries = 10 + while True: + node_id = self._client.least_loaded_node() + for ready_retry in range(40): + if self._client.ready(node_id, False): + break + time.sleep(.1) + else: + raise RuntimeError('Could not connect to broker with node id %d' % (node_id,)) + + try: + future = self._client.send(node_id, request) + future.error_on_callbacks = True + future.add_errback(_failure) + return self._client.poll(future=future, timeout_ms=timeout) + except Exception as exc: + time.sleep(1) + retries -= 1 + if retries == 0: + raise exc + else: + pass # retry + + def _create_topic(self, topic_name, num_partitions, replication_factor, timeout_ms=10000): + if num_partitions is None: + num_partitions = self.partitions + if replication_factor is None: + replication_factor = self.replicas + + # Try different methods to create a topic, from the fastest to the slowest + if self.auto_create_topic and \ + num_partitions == self.partitions and \ + replication_factor == self.replicas: + self._send_request(MetadataRequest[0]([topic_name])) + elif version() >= (0, 10, 1, 0): + request = CreateTopicsRequest[0]([(topic_name, num_partitions, + replication_factor, [], [])], timeout_ms) + result = self._send_request(request, timeout=timeout_ms) + for topic_result in result[0].topic_error_codes: + error_code = topic_result[1] + if error_code != 0: + raise errors.for_code(error_code) + else: + args = self.kafka_run_class_args('kafka.admin.TopicCommand', + '--zookeeper', '%s:%s/%s' % (self.zookeeper.host, + self.zookeeper.port, + self.zk_chroot), + '--create', + '--topic', topic_name, + '--partitions', self.partitions \ + if num_partitions is None else num_partitions, + '--replication-factor', self.replicas \ + if replication_factor is None \ + else replication_factor) + if version() >= (0, 10): + args.append('--if-not-exists') + env = self.kafka_run_class_env() + proc = subprocess.Popen(args, env=env, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + ret = proc.wait() + if ret != 0 or proc.returncode != 0: + output = proc.stdout.read() + if not 'kafka.common.TopicExistsException' in output: + self.out("Failed to create topic %s" % (topic_name,)) + self.out(output) + self.out(proc.stderr.read()) + raise RuntimeError("Failed to create topic %s" % (topic_name,)) + + def create_topics(self, topic_names, num_partitions=None, replication_factor=None): + for topic_name in topic_names: + self._create_topic(topic_name, num_partitions, replication_factor) + + def get_clients(self, cnt=1, client_id=None): + if client_id is None: + client_id = 'client' + return tuple(KafkaClient(client_id='%s_%s' % (client_id, random_string(4)), + bootstrap_servers=self.bootstrap_server()) for x in range(cnt)) + + def get_consumers(self, cnt, topics, **params): + params.setdefault('client_id', 'consumer') + params.setdefault('heartbeat_interval_ms', 500) + params['bootstrap_servers'] = self.bootstrap_server() + client_id = params['client_id'] + for x in range(cnt): + params['client_id'] = '%s_%s' % (client_id, random_string(4)) + yield KafkaConsumer(*topics, **params) + + def get_producers(self, cnt, **params): + params.setdefault('client_id', 'producer') + params['bootstrap_servers'] = self.bootstrap_server() + client_id = params['client_id'] + for x in range(cnt): + params['client_id'] = '%s_%s' % (client_id, random_string(4)) + yield KafkaProducer(**params) + + def get_simple_client(self, **params): + params.setdefault('client_id', 'simple_client') + return SimpleClient(self.bootstrap_server(), **params) diff --git a/test/record/test_default_records.py b/test/record/test_default_records.py index 193703e40..c3a7b02c8 100644 --- a/test/record/test_default_records.py +++ b/test/record/test_default_records.py @@ -1,9 +1,12 @@ # -*- coding: utf-8 -*- from __future__ import unicode_literals import pytest +from mock import patch +import kafka.codec from kafka.record.default_records import ( DefaultRecordBatch, DefaultRecordBatchBuilder ) +from kafka.errors import UnsupportedCodecError @pytest.mark.parametrize("compression_type", [ @@ -17,7 +20,7 @@ def test_read_write_serde_v2(compression_type): magic=2, compression_type=compression_type, is_transactional=1, producer_id=123456, producer_epoch=123, base_sequence=9999, batch_size=999999) - headers = [] # [("header1", b"aaa"), ("header2", b"bbb")] + headers = [("header1", b"aaa"), ("header2", b"bbb")] for offset in range(10): builder.append( offset, timestamp=9999999, key=b"test", value=b"Super", @@ -116,8 +119,12 @@ def test_default_batch_builder_validates_arguments(): builder.append( 5, timestamp=9999999, key=b"123", value=None, headers=[]) + # Check record with headers + builder.append( + 6, timestamp=9999999, key=b"234", value=None, headers=[("hkey", b"hval")]) + # in case error handling code fails to fix inner buffer in builder - assert len(builder.build()) == 104 + assert len(builder.build()) == 124 def test_default_correct_metadata_response(): @@ -167,3 +174,35 @@ def test_default_batch_size_limit(): 2, timestamp=None, key=None, value=b"M" * 700, headers=[]) assert meta is None assert len(builder.build()) < 1000 + + +@pytest.mark.parametrize("compression_type,name,checker_name", [ + (DefaultRecordBatch.CODEC_GZIP, "gzip", "has_gzip"), + (DefaultRecordBatch.CODEC_SNAPPY, "snappy", "has_snappy"), + (DefaultRecordBatch.CODEC_LZ4, "lz4", "has_lz4") +]) +@pytest.mark.parametrize("magic", [0, 1]) +def test_unavailable_codec(magic, compression_type, name, checker_name): + builder = DefaultRecordBatchBuilder( + magic=2, compression_type=compression_type, is_transactional=0, + producer_id=-1, producer_epoch=-1, base_sequence=-1, + batch_size=1024) + builder.append(0, timestamp=None, key=None, value=b"M" * 2000, headers=[]) + correct_buffer = builder.build() + + with patch.object(kafka.codec, checker_name) as mocked: + mocked.return_value = False + # Check that builder raises error + builder = DefaultRecordBatchBuilder( + magic=2, compression_type=compression_type, is_transactional=0, + producer_id=-1, producer_epoch=-1, base_sequence=-1, + batch_size=1024) + error_msg = "Libraries for {} compression codec not found".format(name) + with pytest.raises(UnsupportedCodecError, match=error_msg): + builder.append(0, timestamp=None, key=None, value=b"M", headers=[]) + builder.build() + + # Check that reader raises same error + batch = DefaultRecordBatch(bytes(correct_buffer)) + with pytest.raises(UnsupportedCodecError, match=error_msg): + list(batch) diff --git a/test/record/test_legacy_records.py b/test/record/test_legacy_records.py index ffe8a35f8..23b863605 100644 --- a/test/record/test_legacy_records.py +++ b/test/record/test_legacy_records.py @@ -1,8 +1,11 @@ from __future__ import unicode_literals import pytest +from mock import patch from kafka.record.legacy_records import ( LegacyRecordBatch, LegacyRecordBatchBuilder ) +import kafka.codec +from kafka.errors import UnsupportedCodecError @pytest.mark.parametrize("magic", [0, 1]) @@ -164,3 +167,31 @@ def test_legacy_batch_size_limit(magic): meta = builder.append(2, timestamp=None, key=None, value=b"M" * 700) assert meta is None assert len(builder.build()) < 1000 + + +@pytest.mark.parametrize("compression_type,name,checker_name", [ + (LegacyRecordBatch.CODEC_GZIP, "gzip", "has_gzip"), + (LegacyRecordBatch.CODEC_SNAPPY, "snappy", "has_snappy"), + (LegacyRecordBatch.CODEC_LZ4, "lz4", "has_lz4") +]) +@pytest.mark.parametrize("magic", [0, 1]) +def test_unavailable_codec(magic, compression_type, name, checker_name): + builder = LegacyRecordBatchBuilder( + magic=magic, compression_type=compression_type, batch_size=1024) + builder.append(0, timestamp=None, key=None, value=b"M") + correct_buffer = builder.build() + + with patch.object(kafka.codec, checker_name) as mocked: + mocked.return_value = False + # Check that builder raises error + builder = LegacyRecordBatchBuilder( + magic=magic, compression_type=compression_type, batch_size=1024) + error_msg = "Libraries for {} compression codec not found".format(name) + with pytest.raises(UnsupportedCodecError, match=error_msg): + builder.append(0, timestamp=None, key=None, value=b"M") + builder.build() + + # Check that reader raises same error + batch = LegacyRecordBatch(bytes(correct_buffer), magic) + with pytest.raises(UnsupportedCodecError, match=error_msg): + list(batch) diff --git a/test/record/test_records.py b/test/record/test_records.py index 7306bbc52..f1b8baa40 100644 --- a/test/record/test_records.py +++ b/test/record/test_records.py @@ -1,5 +1,7 @@ +# -*- coding: utf-8 -*- +from __future__ import unicode_literals import pytest -from kafka.record import MemoryRecords +from kafka.record import MemoryRecords, MemoryRecordsBuilder from kafka.errors import CorruptRecordException # This is real live data from Kafka 11 broker @@ -20,6 +22,11 @@ b'\x85\xb7\x00\x00\x00\x00\x00\x00\x00\x00\x01]\xff|\xe7\x9d\x00\x00\x01]' b'\xff|\xe7\x9d\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff' b'\x00\x00\x00\x01\x12\x00\x00\x00\x01\x06123\x00' + # Fourth batch value = "hdr" with header hkey=hval + b'\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00E\x00\x00\x00\x00\x02\\' + b'\xd8\xefR\x00\x00\x00\x00\x00\x00\x00\x00\x01e\x85\xb6\xf3\xc1\x00\x00' + b'\x01e\x85\xb6\xf3\xc1\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff' + b'\xff\xff\x00\x00\x00\x01&\x00\x00\x00\x01\x06hdr\x02\x08hkey\x08hval' ] record_batch_data_v1 = [ @@ -58,8 +65,8 @@ def test_memory_records_v2(): data_bytes = b"".join(record_batch_data_v2) + b"\x00" * 4 records = MemoryRecords(data_bytes) - assert records.size_in_bytes() == 222 - assert records.valid_bytes() == 218 + assert records.size_in_bytes() == 303 + assert records.valid_bytes() == 299 assert records.has_next() is True batch = records.next_batch() @@ -75,6 +82,12 @@ def test_memory_records_v2(): assert records.next_batch() is not None assert records.next_batch() is not None + batch = records.next_batch() + recs = list(batch) + assert len(recs) == 1 + assert recs[0].value == b"hdr" + assert recs[0].headers == [('hkey', b'hval')] + assert records.has_next() is False assert records.next_batch() is None assert records.next_batch() is None @@ -152,3 +165,68 @@ def test_memory_records_corrupt(): ) with pytest.raises(CorruptRecordException): records.next_batch() + + +@pytest.mark.parametrize("compression_type", [0, 1, 2, 3]) +@pytest.mark.parametrize("magic", [0, 1, 2]) +def test_memory_records_builder(magic, compression_type): + builder = MemoryRecordsBuilder( + magic=magic, compression_type=compression_type, batch_size=1024 * 10) + base_size = builder.size_in_bytes() # V2 has a header before + + msg_sizes = [] + for offset in range(10): + metadata = builder.append( + timestamp=10000 + offset, key=b"test", value=b"Super") + msg_sizes.append(metadata.size) + assert metadata.offset == offset + if magic > 0: + assert metadata.timestamp == 10000 + offset + else: + assert metadata.timestamp == -1 + assert builder.next_offset() == offset + 1 + + # Error appends should not leave junk behind, like null bytes or something + with pytest.raises(TypeError): + builder.append( + timestamp=None, key="test", value="Super") # Not bytes, but str + + assert not builder.is_full() + size_before_close = builder.size_in_bytes() + assert size_before_close == sum(msg_sizes) + base_size + + # Size should remain the same after closing. No traling bytes + builder.close() + assert builder.compression_rate() > 0 + expected_size = size_before_close * builder.compression_rate() + assert builder.is_full() + assert builder.size_in_bytes() == expected_size + buffer = builder.buffer() + assert len(buffer) == expected_size + + # We can close second time, as in retry + builder.close() + assert builder.size_in_bytes() == expected_size + assert builder.buffer() == buffer + + # Can't append after close + meta = builder.append(timestamp=None, key=b"test", value=b"Super") + assert meta is None + + +@pytest.mark.parametrize("compression_type", [0, 1, 2, 3]) +@pytest.mark.parametrize("magic", [0, 1, 2]) +def test_memory_records_builder_full(magic, compression_type): + builder = MemoryRecordsBuilder( + magic=magic, compression_type=compression_type, batch_size=1024 * 10) + + # 1 message should always be appended + metadata = builder.append( + key=None, timestamp=None, value=b"M" * 10240) + assert metadata is not None + assert builder.is_full() + + metadata = builder.append( + key=None, timestamp=None, value=b"M") + assert metadata is None + assert builder.next_offset() == 1 diff --git a/test/record/test_util.py b/test/record/test_util.py index bfe0fcc2e..0b2782e7a 100644 --- a/test/record/test_util.py +++ b/test/record/test_util.py @@ -68,9 +68,10 @@ def test_size_of_varint(encoded, decoded): assert util.size_of_varint(decoded) == len(encoded) -def test_crc32c(): +@pytest.mark.parametrize("crc32_func", [util.crc32c_c, util.crc32c_py]) +def test_crc32c(crc32_func): def make_crc(data): - crc = util.calc_crc32c(data) + crc = crc32_func(data) return struct.pack(">I", crc) assert make_crc(b"") == b"\x00\x00\x00\x00" assert make_crc(b"a") == b"\xc1\xd0\x43\x30" diff --git a/test/test_client.py b/test/test_client.py index d02c621a2..c53983c94 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -8,7 +8,7 @@ from kafka import SimpleClient from kafka.errors import ( KafkaUnavailableError, LeaderNotAvailableError, KafkaTimeoutError, - UnknownTopicOrPartitionError, ConnectionError, FailedPayloadsError) + UnknownTopicOrPartitionError, FailedPayloadsError) from kafka.future import Future from kafka.protocol import KafkaProtocol, create_message from kafka.protocol.metadata import MetadataResponse diff --git a/test/test_client_async.py b/test/test_client_async.py index eece139da..09781ac2c 100644 --- a/test/test_client_async.py +++ b/test/test_client_async.py @@ -13,14 +13,13 @@ import pytest from kafka.client_async import KafkaClient, IdleConnectionManager +from kafka.cluster import ClusterMetadata from kafka.conn import ConnectionStates import kafka.errors as Errors from kafka.future import Future from kafka.protocol.metadata import MetadataResponse, MetadataRequest from kafka.protocol.produce import ProduceRequest from kafka.structs import BrokerMetadata -from kafka.cluster import ClusterMetadata -from kafka.future import Future @pytest.fixture @@ -55,21 +54,22 @@ def test_bootstrap_success(conn): kwargs.pop('state_change_callback') kwargs.pop('node_id') assert kwargs == cli.config - conn.connect.assert_called_with() + conn.connect_blocking.assert_called_with() conn.send.assert_called_once_with(MetadataRequest[0]([])) assert cli._bootstrap_fails == 0 assert cli.cluster.brokers() == set([BrokerMetadata(0, 'foo', 12, None), BrokerMetadata(1, 'bar', 34, None)]) + def test_bootstrap_failure(conn): - conn.state = ConnectionStates.DISCONNECTED + conn.connect_blocking.return_value = False cli = KafkaClient(api_version=(0, 9)) args, kwargs = conn.call_args assert args == ('localhost', 9092, socket.AF_UNSPEC) kwargs.pop('state_change_callback') kwargs.pop('node_id') assert kwargs == cli.config - conn.connect.assert_called_with() + conn.connect_blocking.assert_called_with() conn.close.assert_called_with() assert cli._bootstrap_fails == 1 assert cli.cluster.brokers() == set() @@ -95,6 +95,7 @@ def test_can_connect(cli, conn): conn.blacked_out.return_value = True assert not cli._can_connect(0) + def test_maybe_connect(cli, conn): try: # Node not in metadata, raises AssertionError diff --git a/test/test_client_integration.py b/test/test_client_integration.py index 742572d5e..df0faef69 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -17,7 +17,7 @@ def setUpClass(cls): # noqa return cls.zk = ZookeeperFixture.instance() - cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) + cls.server = KafkaFixture.instance(0, cls.zk) @classmethod def tearDownClass(cls): # noqa diff --git a/test/test_cluster.py b/test/test_cluster.py new file mode 100644 index 000000000..f010c4f71 --- /dev/null +++ b/test/test_cluster.py @@ -0,0 +1,22 @@ +# pylint: skip-file +from __future__ import absolute_import + +import pytest + +from kafka.cluster import ClusterMetadata +from kafka.protocol.metadata import MetadataResponse + + +def test_empty_broker_list(): + cluster = ClusterMetadata() + assert len(cluster.brokers()) == 0 + + cluster.update_metadata(MetadataResponse[0]( + [(0, 'foo', 12), (1, 'bar', 34)], [])) + assert len(cluster.brokers()) == 2 + + # empty broker list response should be ignored + cluster.update_metadata(MetadataResponse[0]( + [], # empty brokers + [(17, 'foo', []), (17, 'bar', [])])) # topics w/ error + assert len(cluster.brokers()) == 2 diff --git a/test/test_conn.py b/test/test_conn.py index 56985190a..27d77beb3 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -13,7 +13,7 @@ from kafka.protocol.metadata import MetadataRequest from kafka.protocol.produce import ProduceRequest -import kafka.common as Errors +import kafka.errors as Errors @pytest.fixture @@ -72,6 +72,15 @@ def test_blacked_out(conn): assert conn.blacked_out() is True +def test_connection_delay(conn): + conn.last_attempt = time.time() + assert round(conn.connection_delay()) == round(conn.config['reconnect_backoff_ms']) + conn.state = ConnectionStates.CONNECTING + assert conn.connection_delay() == 0 + conn.state = ConnectionStates.CONNECTED + assert conn.connection_delay() == float('inf') + + def test_connected(conn): assert conn.connected() is False conn.state = ConnectionStates.CONNECTED @@ -90,7 +99,7 @@ def test_send_disconnected(conn): conn.state = ConnectionStates.DISCONNECTED f = conn.send('foobar') assert f.failed() is True - assert isinstance(f.exception, Errors.ConnectionError) + assert isinstance(f.exception, Errors.KafkaConnectionError) def test_send_connecting(conn): @@ -153,7 +162,7 @@ def test_send_error(_socket, conn): _socket.send.side_effect = socket.error f = conn.send(req) assert f.failed() is True - assert isinstance(f.exception, Errors.ConnectionError) + assert isinstance(f.exception, Errors.KafkaConnectionError) assert _socket.close.call_count == 1 assert conn.state is ConnectionStates.DISCONNECTED @@ -246,35 +255,41 @@ def test_lookup_on_connect(): hostname = 'example.org' port = 9092 conn = BrokerConnection(hostname, port, socket.AF_UNSPEC) - assert conn.host == conn.hostname == hostname - ip1 = '127.0.0.1' + assert conn.host == hostname + assert conn.port == port + assert conn.afi == socket.AF_UNSPEC + afi1 = socket.AF_INET + sockaddr1 = ('127.0.0.1', 9092) mock_return1 = [ - (2, 2, 17, '', (ip1, 9092)), + (afi1, socket.SOCK_STREAM, 6, '', sockaddr1), ] with mock.patch("socket.getaddrinfo", return_value=mock_return1) as m: conn.connect() m.assert_called_once_with(hostname, port, 0, 1) + assert conn._sock_afi == afi1 + assert conn._sock_addr == sockaddr1 conn.close() - assert conn.host == ip1 - ip2 = '127.0.0.2' + afi2 = socket.AF_INET6 + sockaddr2 = ('::1', 9092, 0, 0) mock_return2 = [ - (2, 2, 17, '', (ip2, 9092)), + (afi2, socket.SOCK_STREAM, 6, '', sockaddr2), ] with mock.patch("socket.getaddrinfo", return_value=mock_return2) as m: conn.last_attempt = 0 conn.connect() m.assert_called_once_with(hostname, port, 0, 1) + assert conn._sock_afi == afi2 + assert conn._sock_addr == sockaddr2 conn.close() - assert conn.host == ip2 def test_relookup_on_failure(): hostname = 'example.org' port = 9092 conn = BrokerConnection(hostname, port, socket.AF_UNSPEC) - assert conn.host == conn.hostname == hostname + assert conn.host == hostname mock_return1 = [] with mock.patch("socket.getaddrinfo", return_value=mock_return1) as m: last_attempt = conn.last_attempt @@ -283,14 +298,16 @@ def test_relookup_on_failure(): assert conn.disconnected() assert conn.last_attempt > last_attempt - ip2 = '127.0.0.2' + afi2 = socket.AF_INET + sockaddr2 = ('127.0.0.2', 9092) mock_return2 = [ - (2, 2, 17, '', (ip2, 9092)), + (afi2, socket.SOCK_STREAM, 6, '', sockaddr2), ] with mock.patch("socket.getaddrinfo", return_value=mock_return2) as m: conn.last_attempt = 0 conn.connect() m.assert_called_once_with(hostname, port, 0, 1) + assert conn._sock_afi == afi2 + assert conn._sock_addr == sockaddr2 conn.close() - assert conn.host == ip2 diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index b9307486e..f9a41a46a 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -44,6 +44,7 @@ def test_consumer(kafka_broker, version): assert len(consumer._client._conns) > 0 node_id = list(consumer._client._conns.keys())[0] assert consumer._client._conns[node_id].state is ConnectionStates.CONNECTED + consumer.close() @pytest.mark.skipif(version() < (0, 9), reason='Unsupported Kafka Version') @@ -153,6 +154,7 @@ def test_paused(kafka_broker, topic): consumer.unsubscribe() assert set() == consumer.paused() + consumer.close() @pytest.mark.skipif(version() < (0, 9), reason='Unsupported Kafka Version') @@ -183,3 +185,4 @@ def test_heartbeat_thread(kafka_broker, topic): assert consumer._coordinator.heartbeat.last_poll == last_poll consumer.poll(timeout_ms=100) assert consumer._coordinator.heartbeat.last_poll > last_poll + consumer.close() diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index ded231477..e6f140598 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -1,7 +1,11 @@ import logging import os import time +from mock import patch +import pytest +import kafka.codec +import pytest from six.moves import xrange import six @@ -13,18 +17,61 @@ from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES from kafka.errors import ( ConsumerFetchSizeTooSmall, OffsetOutOfRangeError, UnsupportedVersionError, - KafkaTimeoutError + KafkaTimeoutError, UnsupportedCodecError ) from kafka.structs import ( ProduceRequestPayload, TopicPartition, OffsetAndTimestamp ) +from test.conftest import version from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( - KafkaIntegrationTestCase, kafka_versions, random_string, Timer + KafkaIntegrationTestCase, kafka_versions, random_string, Timer, + send_messages ) +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_kafka_consumer(simple_client, topic, kafka_consumer_factory): + """Test KafkaConsumer + """ + kafka_consumer = kafka_consumer_factory(auto_offset_reset='earliest') + + send_messages(simple_client, topic, 0, range(0, 100)) + send_messages(simple_client, topic, 1, range(100, 200)) + + cnt = 0 + messages = {0: set(), 1: set()} + for message in kafka_consumer: + logging.debug("Consumed message %s", repr(message)) + cnt += 1 + messages[message.partition].add(message.offset) + if cnt >= 200: + break + + assert len(messages[0]) == 100 + assert len(messages[1]) == 100 + kafka_consumer.close() + + +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_kafka_consumer_unsupported_encoding( + topic, kafka_producer_factory, kafka_consumer_factory): + # Send a compressed message + producer = kafka_producer_factory(compression_type="gzip") + fut = producer.send(topic, b"simple message" * 200) + fut.get(timeout=5) + producer.close() + + # Consume, but with the related compression codec not available + with patch.object(kafka.codec, "has_gzip") as mocked: + mocked.return_value = False + consumer = kafka_consumer_factory(auto_offset_reset='earliest') + error_msg = "Libraries for gzip compression codec not found" + with pytest.raises(UnsupportedCodecError, match=error_msg): + consumer.poll(timeout_ms=2000) + + class TestConsumerIntegration(KafkaIntegrationTestCase): maxDiff = None @@ -35,9 +82,9 @@ def setUpClass(cls): cls.zk = ZookeeperFixture.instance() chroot = random_string(10) - cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port, + cls.server1 = KafkaFixture.instance(0, cls.zk, zk_chroot=chroot) - cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port, + cls.server2 = KafkaFixture.instance(1, cls.zk, zk_chroot=chroot) cls.server = cls.server1 # Bootstrapping server @@ -501,24 +548,6 @@ def test_fetch_buffer_size(self): messages = [ message for message in consumer ] self.assertEqual(len(messages), 2) - def test_kafka_consumer(self): - self.send_messages(0, range(0, 100)) - self.send_messages(1, range(100, 200)) - - # Start a consumer - consumer = self.kafka_consumer(auto_offset_reset='earliest') - n = 0 - messages = {0: set(), 1: set()} - for m in consumer: - logging.debug("Consumed message %s" % repr(m)) - n += 1 - messages[m.partition].add(m.offset) - if n >= 200: - break - - self.assertEqual(len(messages[0]), 100) - self.assertEqual(len(messages[1]), 100) - def test_kafka_consumer__blocking(self): TIMEOUT_MS = 500 consumer = self.kafka_consumer(auto_offset_reset='earliest', @@ -555,6 +584,7 @@ def test_kafka_consumer__blocking(self): messages.add((msg.partition, msg.offset)) self.assertEqual(len(messages), 5) self.assertGreaterEqual(t.interval, TIMEOUT_MS / 1000.0 ) + consumer.close() @kafka_versions('>=0.8.1') def test_kafka_consumer__offset_commit_resume(self): @@ -594,6 +624,7 @@ def test_kafka_consumer__offset_commit_resume(self): output_msgs2.append(m) self.assert_message_count(output_msgs2, 20) self.assertEqual(len(set(output_msgs1) | set(output_msgs2)), 200) + consumer2.close() @kafka_versions('>=0.10.1') def test_kafka_consumer_max_bytes_simple(self): @@ -614,6 +645,7 @@ def test_kafka_consumer_max_bytes_simple(self): self.assertEqual( seen_partitions, set([ TopicPartition(self.topic, 0), TopicPartition(self.topic, 1)])) + consumer.close() @kafka_versions('>=0.10.1') def test_kafka_consumer_max_bytes_one_msg(self): @@ -639,6 +671,7 @@ def test_kafka_consumer_max_bytes_one_msg(self): fetched_msgs = [next(consumer) for i in range(10)] self.assertEqual(len(fetched_msgs), 10) + consumer.close() @kafka_versions('>=0.10.1') def test_kafka_consumer_offsets_for_time(self): @@ -647,13 +680,14 @@ def test_kafka_consumer_offsets_for_time(self): early_time = late_time - 2000 tp = TopicPartition(self.topic, 0) + timeout = 10 kafka_producer = self.kafka_producer() early_msg = kafka_producer.send( self.topic, partition=0, value=b"first", - timestamp_ms=early_time).get(1) + timestamp_ms=early_time).get(timeout) late_msg = kafka_producer.send( self.topic, partition=0, value=b"last", - timestamp_ms=late_time).get(1) + timestamp_ms=late_time).get(timeout) consumer = self.kafka_consumer() offsets = consumer.offsets_for_times({tp: early_time}) @@ -691,6 +725,7 @@ def test_kafka_consumer_offsets_for_time(self): self.assertEqual(offsets, { tp: late_msg.offset + 1 }) + consumer.close() @kafka_versions('>=0.10.1') def test_kafka_consumer_offsets_search_many_partitions(self): @@ -699,12 +734,13 @@ def test_kafka_consumer_offsets_search_many_partitions(self): kafka_producer = self.kafka_producer() send_time = int(time.time() * 1000) + timeout = 10 p0msg = kafka_producer.send( self.topic, partition=0, value=b"XXX", - timestamp_ms=send_time).get() + timestamp_ms=send_time).get(timeout) p1msg = kafka_producer.send( self.topic, partition=1, value=b"XXX", - timestamp_ms=send_time).get() + timestamp_ms=send_time).get(timeout) consumer = self.kafka_consumer() offsets = consumer.offsets_for_times({ @@ -728,6 +764,7 @@ def test_kafka_consumer_offsets_search_many_partitions(self): tp0: p0msg.offset + 1, tp1: p1msg.offset + 1 }) + consumer.close() @kafka_versions('<0.10.1') def test_kafka_consumer_offsets_for_time_old(self): diff --git a/test/test_coordinator.py b/test/test_coordinator.py index e094b9c8e..4afdcd9ac 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -5,7 +5,6 @@ import pytest from kafka.client_async import KafkaClient -from kafka.structs import TopicPartition, OffsetAndMetadata from kafka.consumer.subscription_state import ( SubscriptionState, ConsumerRebalanceListener) from kafka.coordinator.assignors.range import RangePartitionAssignor @@ -21,6 +20,7 @@ OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse) from kafka.protocol.metadata import MetadataResponse +from kafka.structs import TopicPartition, OffsetAndMetadata from kafka.util import WeakMethod @@ -34,7 +34,7 @@ def coordinator(client): def test_init(client, coordinator): - # metadata update on init + # metadata update on init assert client.cluster._need_update is True assert WeakMethod(coordinator._handle_metadata_update) in client.cluster._listeners @@ -62,7 +62,7 @@ def test_group_protocols(coordinator): # Requires a subscription try: coordinator.group_protocols() - except AssertionError: + except Errors.IllegalStateError: pass else: assert False, 'Exception not raised when expected' @@ -85,8 +85,7 @@ def test_pattern_subscription(coordinator, api_version): coordinator.config['api_version'] = api_version coordinator._subscription.subscribe(pattern='foo') assert coordinator._subscription.subscription == set([]) - assert coordinator._subscription_metadata_changed({}) is False - assert coordinator._subscription.needs_partition_assignment is False + assert coordinator._metadata_snapshot == coordinator._build_metadata_snapshot(coordinator._subscription, {}) cluster = coordinator._client.cluster cluster.update_metadata(MetadataResponse[0]( @@ -100,12 +99,10 @@ def test_pattern_subscription(coordinator, api_version): # 0.9 consumers should trigger dynamic partition assignment if api_version >= (0, 9): - assert coordinator._subscription.needs_partition_assignment is True assert coordinator._subscription.assignment == {} # earlier consumers get all partitions assigned locally else: - assert coordinator._subscription.needs_partition_assignment is False assert set(coordinator._subscription.assignment.keys()) == set([ TopicPartition('foo1', 0), TopicPartition('foo2', 0)]) @@ -195,7 +192,6 @@ def test_perform_assignment(mocker, coordinator): def test_on_join_prepare(coordinator): coordinator._subscription.subscribe(topics=['foobar']) coordinator._on_join_prepare(0, 'member-foo') - assert coordinator._subscription.needs_partition_assignment is True def test_need_rejoin(coordinator): @@ -205,13 +201,6 @@ def test_need_rejoin(coordinator): coordinator._subscription.subscribe(topics=['foobar']) assert coordinator.need_rejoin() is True - coordinator._subscription.needs_partition_assignment = False - coordinator.rejoin_needed = False - assert coordinator.need_rejoin() is False - - coordinator._subscription.needs_partition_assignment = True - assert coordinator.need_rejoin() is True - def test_refresh_committed_offsets_if_needed(mocker, coordinator): mocker.patch.object(ConsumerCoordinator, 'fetch_committed_offsets', @@ -388,7 +377,6 @@ def test_maybe_auto_commit_offsets_sync(mocker, api_version, group_id, enable, @pytest.fixture def patched_coord(mocker, coordinator): coordinator._subscription.subscribe(topics=['foobar']) - coordinator._subscription.needs_partition_assignment = False mocker.patch.object(coordinator, 'coordinator_unknown', return_value=False) coordinator.coordinator_id = 0 mocker.patch.object(coordinator, 'coordinator', return_value=0) @@ -461,47 +449,39 @@ def test_send_offset_commit_request_success(mocker, patched_coord, offsets): offsets, future, mocker.ANY, response) -@pytest.mark.parametrize('response,error,dead,reassign', [ +@pytest.mark.parametrize('response,error,dead', [ (OffsetCommitResponse[0]([('foobar', [(0, 30), (1, 30)])]), - Errors.GroupAuthorizationFailedError, False, False), + Errors.GroupAuthorizationFailedError, False), (OffsetCommitResponse[0]([('foobar', [(0, 12), (1, 12)])]), - Errors.OffsetMetadataTooLargeError, False, False), + Errors.OffsetMetadataTooLargeError, False), (OffsetCommitResponse[0]([('foobar', [(0, 28), (1, 28)])]), - Errors.InvalidCommitOffsetSizeError, False, False), + Errors.InvalidCommitOffsetSizeError, False), (OffsetCommitResponse[0]([('foobar', [(0, 14), (1, 14)])]), - Errors.GroupLoadInProgressError, False, False), + Errors.GroupLoadInProgressError, False), (OffsetCommitResponse[0]([('foobar', [(0, 15), (1, 15)])]), - Errors.GroupCoordinatorNotAvailableError, True, False), + Errors.GroupCoordinatorNotAvailableError, True), (OffsetCommitResponse[0]([('foobar', [(0, 16), (1, 16)])]), - Errors.NotCoordinatorForGroupError, True, False), + Errors.NotCoordinatorForGroupError, True), (OffsetCommitResponse[0]([('foobar', [(0, 7), (1, 7)])]), - Errors.RequestTimedOutError, True, False), + Errors.RequestTimedOutError, True), (OffsetCommitResponse[0]([('foobar', [(0, 25), (1, 25)])]), - Errors.CommitFailedError, False, True), + Errors.CommitFailedError, False), (OffsetCommitResponse[0]([('foobar', [(0, 22), (1, 22)])]), - Errors.CommitFailedError, False, True), + Errors.CommitFailedError, False), (OffsetCommitResponse[0]([('foobar', [(0, 27), (1, 27)])]), - Errors.CommitFailedError, False, True), + Errors.CommitFailedError, False), (OffsetCommitResponse[0]([('foobar', [(0, 17), (1, 17)])]), - Errors.InvalidTopicError, False, False), + Errors.InvalidTopicError, False), (OffsetCommitResponse[0]([('foobar', [(0, 29), (1, 29)])]), - Errors.TopicAuthorizationFailedError, False, False), + Errors.TopicAuthorizationFailedError, False), ]) def test_handle_offset_commit_response(mocker, patched_coord, offsets, - response, error, dead, reassign): + response, error, dead): future = Future() patched_coord._handle_offset_commit_response(offsets, future, time.time(), response) assert isinstance(future.exception, error) assert patched_coord.coordinator_id is (None if dead else 0) - if reassign: - assert patched_coord._generation is Generation.NO_GENERATION - assert patched_coord.rejoin_needed is True - assert patched_coord.state is MemberState.UNJOINED - else: - assert patched_coord._generation is not Generation.NO_GENERATION - assert patched_coord.rejoin_needed is False - assert patched_coord.state is MemberState.STABLE @pytest.fixture @@ -562,7 +542,7 @@ def test_send_offset_fetch_request_success(patched_coord, partitions): response = OffsetFetchResponse[0]([('foobar', [(0, 123, b'', 0), (1, 234, b'', 0)])]) _f.success(response) patched_coord._handle_offset_fetch_response.assert_called_with( - future, response) + future, response) @pytest.mark.parametrize('response,error,dead', [ @@ -570,6 +550,10 @@ def test_send_offset_fetch_request_success(patched_coord, partitions): Errors.GroupLoadInProgressError, False), (OffsetFetchResponse[0]([('foobar', [(0, 123, b'', 16), (1, 234, b'', 16)])]), Errors.NotCoordinatorForGroupError, True), + (OffsetFetchResponse[0]([('foobar', [(0, 123, b'', 25), (1, 234, b'', 25)])]), + Errors.UnknownMemberIdError, False), + (OffsetFetchResponse[0]([('foobar', [(0, 123, b'', 22), (1, 234, b'', 22)])]), + Errors.IllegalGenerationError, False), (OffsetFetchResponse[0]([('foobar', [(0, 123, b'', 29), (1, 234, b'', 29)])]), Errors.TopicAuthorizationFailedError, False), (OffsetFetchResponse[0]([('foobar', [(0, 123, b'', 0), (1, 234, b'', 0)])]), @@ -627,7 +611,7 @@ def test_ensure_active_group(mocker, coordinator): coordinator._subscription.subscribe(topics=['foobar']) mocker.patch.object(coordinator, 'coordinator_unknown', return_value=False) mocker.patch.object(coordinator, '_send_join_group_request', return_value=Future().success(True)) - mocker.patch.object(coordinator, 'need_rejoin', side_effect=[True, True, False]) + mocker.patch.object(coordinator, 'need_rejoin', side_effect=[True, False]) mocker.patch.object(coordinator, '_on_join_complete') mocker.patch.object(coordinator, '_heartbeat_thread') diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 9141947ac..ad7dcb98b 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -4,7 +4,7 @@ from kafka import SimpleClient, SimpleConsumer, KeyedProducer from kafka.errors import ( - FailedPayloadsError, ConnectionError, RequestTimedOutError, + FailedPayloadsError, KafkaConnectionError, RequestTimedOutError, NotLeaderForPartitionError) from kafka.producer.base import Producer from kafka.structs import TopicPartition @@ -29,10 +29,9 @@ def setUp(self): # mini zookeeper, 3 kafka brokers self.zk = ZookeeperFixture.instance() - kk_args = [self.zk.host, self.zk.port] kk_kwargs = {'zk_chroot': zk_chroot, 'replicas': replicas, 'partitions': partitions} - self.brokers = [KafkaFixture.instance(i, *kk_args, **kk_kwargs) + self.brokers = [KafkaFixture.instance(i, self.zk, **kk_kwargs) for i in range(replicas)] hosts = ['%s:%d' % (b.host, b.port) for b in self.brokers] @@ -61,7 +60,7 @@ def test_switch_leader(self): # require that the server commit messages to all in-sync replicas # so that failover doesn't lose any messages on server-side # and we can assert that server-side message count equals client-side - producer = Producer(self.client, async=False, + producer = Producer(self.client, async_send=False, req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT) # Send 100 random messages to a specific partition @@ -80,7 +79,7 @@ def test_switch_leader(self): producer.send_messages(topic, partition, b'success') log.debug("success!") recovered = True - except (FailedPayloadsError, ConnectionError, RequestTimedOutError, + except (FailedPayloadsError, KafkaConnectionError, RequestTimedOutError, NotLeaderForPartitionError): log.debug("caught exception sending message -- will retry") continue @@ -102,7 +101,7 @@ def test_switch_leader_async(self): partition = 0 # Test the base class Producer -- send_messages to a specific partition - producer = Producer(self.client, async=True, + producer = Producer(self.client, async_send=True, batch_send_every_n=15, batch_send_every_t=3, req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT, @@ -147,7 +146,7 @@ def test_switch_leader_async(self): def test_switch_leader_keyed_producer(self): topic = self.topic - producer = KeyedProducer(self.client, async=False) + producer = KeyedProducer(self.client, async_send=False) # Send 10 random messages for _ in range(10): @@ -168,7 +167,7 @@ def test_switch_leader_keyed_producer(self): producer.send_messages(topic, key, msg) if producer.partitioners[topic].partition(key) == 0: recovered = True - except (FailedPayloadsError, ConnectionError, RequestTimedOutError, + except (FailedPayloadsError, KafkaConnectionError, RequestTimedOutError, NotLeaderForPartitionError): log.debug("caught exception sending message -- will retry") continue @@ -183,7 +182,7 @@ def test_switch_leader_keyed_producer(self): producer.send_messages(topic, key, msg) def test_switch_leader_simple_consumer(self): - producer = Producer(self.client, async=False) + producer = Producer(self.client, async_send=False) consumer = SimpleConsumer(self.client, None, self.topic, partitions=None, auto_commit=False, iter_timeout=10) self._send_random_messages(producer, self.topic, 0, 2) consumer.get_messages() diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 429071a72..e37a70db5 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -12,16 +12,16 @@ CompletedFetch, ConsumerRecord, Fetcher, NoOffsetForPartitionError ) from kafka.consumer.subscription_state import SubscriptionState +from kafka.future import Future from kafka.metrics import Metrics from kafka.protocol.fetch import FetchRequest, FetchResponse from kafka.protocol.offset import OffsetResponse -from kafka.structs import TopicPartition -from kafka.future import Future from kafka.errors import ( StaleMetadata, LeaderNotAvailableError, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError ) from kafka.record.memory_records import MemoryRecordsBuilder, MemoryRecords +from kafka.structs import TopicPartition @pytest.fixture @@ -498,3 +498,60 @@ def test__parse_fetched_data__out_of_range(fetcher, topic, mocker): partition_record = fetcher._parse_fetched_data(completed_fetch) assert partition_record is None assert fetcher._subscriptions.assignment[tp].awaiting_reset is True + + +def test_partition_records_offset(): + """Test that compressed messagesets are handle correctly + when fetch offset is in the middle of the message list + """ + batch_start = 120 + batch_end = 130 + fetch_offset = 123 + tp = TopicPartition('foo', 0) + messages = [ConsumerRecord(tp.topic, tp.partition, i, + None, None, 'key', 'value', [], 'checksum', 0, 0, -1) + for i in range(batch_start, batch_end)] + records = Fetcher.PartitionRecords(fetch_offset, None, messages) + assert len(records) > 0 + msgs = records.take(1) + assert msgs[0].offset == fetch_offset + assert records.fetch_offset == fetch_offset + 1 + msgs = records.take(2) + assert len(msgs) == 2 + assert len(records) > 0 + records.discard() + assert len(records) == 0 + + +def test_partition_records_empty(): + records = Fetcher.PartitionRecords(0, None, []) + assert len(records) == 0 + + +def test_partition_records_no_fetch_offset(): + batch_start = 0 + batch_end = 100 + fetch_offset = 123 + tp = TopicPartition('foo', 0) + messages = [ConsumerRecord(tp.topic, tp.partition, i, + None, None, 'key', 'value', None, 'checksum', 0, 0, -1) + for i in range(batch_start, batch_end)] + records = Fetcher.PartitionRecords(fetch_offset, None, messages) + assert len(records) == 0 + + +def test_partition_records_compacted_offset(): + """Test that messagesets are handle correctly + when the fetch offset points to a message that has been compacted + """ + batch_start = 0 + batch_end = 100 + fetch_offset = 42 + tp = TopicPartition('foo', 0) + messages = [ConsumerRecord(tp.topic, tp.partition, i, + None, None, 'key', 'value', None, 'checksum', 0, 0, -1) + for i in range(batch_start, batch_end) if i != fetch_offset] + records = Fetcher.PartitionRecords(fetch_offset, None, messages) + assert len(records) == batch_end - fetch_offset - 1 + msgs = records.take(1) + assert msgs[0].offset == fetch_offset + 1 diff --git a/test/test_producer.py b/test/test_producer.py index f7a5b68f6..176b23988 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -38,12 +38,12 @@ def test_end_to_end(kafka_broker, compression): connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) producer = KafkaProducer(bootstrap_servers=connect_str, retries=5, - max_block_ms=10000, + max_block_ms=30000, compression_type=compression, value_serializer=str.encode) consumer = KafkaConsumer(bootstrap_servers=connect_str, group_id=None, - consumer_timeout_ms=10000, + consumer_timeout_ms=30000, auto_offset_reset='earliest', value_deserializer=bytes.decode) @@ -55,7 +55,6 @@ def test_end_to_end(kafka_broker, compression): futures.append(producer.send(topic, 'msg %d' % i)) ret = [f.get(timeout=30) for f in futures] assert len(ret) == messages - producer.close() consumer.subscribe([topic]) @@ -67,6 +66,7 @@ def test_end_to_end(kafka_broker, compression): break assert msgs == set(['msg %d' % i for i in range(messages)]) + consumer.close() @pytest.mark.skipif(platform.python_implementation() != 'CPython', @@ -87,14 +87,20 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression): connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) producer = KafkaProducer(bootstrap_servers=connect_str, retries=5, - max_block_ms=10000, + max_block_ms=30000, compression_type=compression) magic = producer._max_usable_produce_magic() + # record headers are supported in 0.11.0 + if version() < (0, 11, 0): + headers = None + else: + headers = [("Header Key", b"Header Value")] + topic = random_string(5) future = producer.send( topic, - value=b"Simple value", key=b"Simple key", timestamp_ms=9999999, + value=b"Simple value", key=b"Simple key", headers=headers, timestamp_ms=9999999, partition=0) record = future.get(timeout=5) assert record is not None @@ -116,6 +122,8 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression): assert record.serialized_key_size == 10 assert record.serialized_value_size == 12 + if headers: + assert record.serialized_header_size == 22 # generated timestamp case is skipped for broker 0.9 and below if magic == 0: diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index a304e83b6..6533cfabb 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -2,6 +2,7 @@ import time import uuid +import pytest from six.moves import range from kafka import ( @@ -14,8 +15,56 @@ from kafka.producer.base import Producer from kafka.structs import FetchRequestPayload, ProduceRequestPayload +from test.conftest import version from test.fixtures import ZookeeperFixture, KafkaFixture -from test.testutil import KafkaIntegrationTestCase, kafka_versions +from test.testutil import KafkaIntegrationTestCase, kafka_versions, current_offset + + +# TODO: This duplicates a TestKafkaProducerIntegration method temporarily +# while the migration to pytest is in progress +def assert_produce_request(client, topic, messages, initial_offset, message_ct, + partition=0): + """Verify the correctness of a produce request + """ + produce = ProduceRequestPayload(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. + resp = client.send_produce_request([produce]) + assert_produce_response(resp, initial_offset) + + assert current_offset(client, topic, partition) == initial_offset + message_ct + + +def assert_produce_response(resp, initial_offset): + """Verify that a produce response is well-formed + """ + assert len(resp) == 1 + assert resp[0].error == 0 + assert resp[0].offset == initial_offset + + +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_produce_many_simple(simple_client, topic): + """Test multiple produces using the SimpleClient + """ + start_offset = current_offset(simple_client, topic, 0) + + assert_produce_request( + simple_client, topic, + [create_message(("Test message %d" % i).encode('utf-8')) + for i in range(100)], + start_offset, + 100, + ) + + assert_produce_request( + simple_client, topic, + [create_message(("Test message %d" % i).encode('utf-8')) + for i in range(100)], + start_offset+100, + 100, + ) class TestKafkaProducerIntegration(KafkaIntegrationTestCase): @@ -26,7 +75,7 @@ def setUpClass(cls): # noqa return cls.zk = ZookeeperFixture.instance() - cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) + cls.server = KafkaFixture.instance(0, cls.zk) @classmethod def tearDownClass(cls): # noqa @@ -36,23 +85,6 @@ def tearDownClass(cls): # noqa cls.server.close() cls.zk.close() - def test_produce_many_simple(self): - start_offset = self.current_offset(self.topic, 0) - - self.assert_produce_request( - [create_message(("Test message %d" % i).encode('utf-8')) - for i in range(100)], - start_offset, - 100, - ) - - self.assert_produce_request( - [create_message(("Test message %d" % i).encode('utf-8')) - for i in range(100)], - start_offset+100, - 100, - ) - def test_produce_10k_simple(self): start_offset = self.current_offset(self.topic, 0) @@ -184,7 +216,7 @@ def test_async_simple_producer(self): partition = self.client.get_partition_ids_for_topic(self.topic)[0] start_offset = self.current_offset(self.topic, partition) - producer = SimpleProducer(self.client, async=True, random_start=False) + producer = SimpleProducer(self.client, async_send=True, random_start=False) resp = producer.send_messages(self.topic, self.msg("one")) self.assertEqual(len(resp), 0) @@ -203,7 +235,7 @@ def test_batched_simple_producer__triggers_by_message(self): batch_interval = 5 producer = SimpleProducer( self.client, - async=True, + async_send=True, batch_send_every_n=batch_messages, batch_send_every_t=batch_interval, random_start=False) @@ -262,13 +294,14 @@ def test_batched_simple_producer__triggers_by_message(self): producer.stop() def test_batched_simple_producer__triggers_by_time(self): + self.skipTest("Flakey test -- should be refactored or removed") partitions = self.client.get_partition_ids_for_topic(self.topic) start_offsets = [self.current_offset(self.topic, p) for p in partitions] batch_interval = 5 producer = SimpleProducer( self.client, - async=True, + async_send=True, batch_send_every_n=100, batch_send_every_t=batch_interval, random_start=False) @@ -400,7 +433,7 @@ def test_async_keyed_producer(self): producer = KeyedProducer(self.client, partitioner=RoundRobinPartitioner, - async=True, + async_send=True, batch_send_every_t=1) resp = producer.send_messages(self.topic, self.key("key1"), self.msg("one")) diff --git a/test/test_producer_legacy.py b/test/test_producer_legacy.py index 9b87c7664..6d00116c3 100644 --- a/test/test_producer_legacy.py +++ b/test/test_producer_legacy.py @@ -73,7 +73,7 @@ def partitions(topic): @patch('kafka.producer.base._send_upstream') def test_producer_async_queue_overfilled(self, mock): queue_size = 2 - producer = Producer(MagicMock(), async=True, + producer = Producer(MagicMock(), async_send=True, async_queue_maxsize=queue_size) topic = b'test-topic' @@ -95,25 +95,25 @@ def test_producer_sync_fail_on_error(self): with patch.object(SimpleClient, '_send_broker_aware_request', return_value = [error]): client = SimpleClient(MagicMock()) - producer = SimpleProducer(client, async=False, sync_fail_on_error=False) + producer = SimpleProducer(client, async_send=False, sync_fail_on_error=False) # This should not raise (response,) = producer.send_messages('foobar', b'test message') self.assertEqual(response, error) - producer = SimpleProducer(client, async=False, sync_fail_on_error=True) + producer = SimpleProducer(client, async_send=False, sync_fail_on_error=True) with self.assertRaises(FailedPayloadsError): producer.send_messages('foobar', b'test message') def test_cleanup_is_not_called_on_stopped_producer(self): - producer = Producer(MagicMock(), async=True) + producer = Producer(MagicMock(), async_send=True) producer.stopped = True with patch.object(producer, 'stop') as mocked_stop: producer._cleanup_func(producer) self.assertEqual(mocked_stop.call_count, 0) def test_cleanup_is_called_on_running_producer(self): - producer = Producer(MagicMock(), async=True) + producer = Producer(MagicMock(), async_send=True) producer.stopped = False with patch.object(producer, 'stop') as mocked_stop: producer._cleanup_func(producer) diff --git a/test/test_util.py b/test/test_util.py index 58e5ab840..fb592e8e6 100644 --- a/test/test_util.py +++ b/test/test_util.py @@ -5,8 +5,8 @@ from . import unittest import kafka.errors -import kafka.util import kafka.structs +import kafka.util class UtilTest(unittest.TestCase): diff --git a/test/testutil.py b/test/testutil.py index 0bacac411..a1383a0a0 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -1,35 +1,26 @@ +from __future__ import absolute_import + import functools -import logging import operator import os -import random import socket -import string import time import uuid -from six.moves import xrange +import pytest from . import unittest -from kafka import SimpleClient -from kafka.structs import OffsetRequestPayload - -__all__ = [ - 'random_string', - 'get_open_port', - 'kafka_versions', - 'KafkaIntegrationTestCase', - 'Timer', -] - -def random_string(l): - return "".join(random.choice(string.ascii_letters) for i in xrange(l)) +from kafka import SimpleClient, create_message +from kafka.errors import ( + LeaderNotAvailableError, KafkaTimeoutError, InvalidTopicError, + NotLeaderForPartitionError, UnknownTopicOrPartitionError, + FailedPayloadsError +) +from kafka.structs import OffsetRequestPayload, ProduceRequestPayload +from test.fixtures import random_string, version_str_to_list, version as kafka_version #pylint: disable=wrong-import-order def kafka_versions(*versions): - def version_str_to_list(s): - return list(map(int, s.split('.'))) # e.g., [0, 8, 1, 1] - def construct_lambda(s): if s[0].isdigit(): op_str = '=' @@ -53,25 +44,26 @@ def construct_lambda(s): } op = op_map[op_str] version = version_str_to_list(v_str) - return lambda a: op(version_str_to_list(a), version) + return lambda a: op(a, version) validators = map(construct_lambda, versions) - def kafka_versions(func): + def real_kafka_versions(func): @functools.wraps(func) - def wrapper(self): - kafka_version = os.environ.get('KAFKA_VERSION') + def wrapper(func, *args, **kwargs): + version = kafka_version() - if not kafka_version: - self.skipTest("no kafka version set in KAFKA_VERSION env var") + if not version: + pytest.skip("no kafka version set in KAFKA_VERSION env var") for f in validators: - if not f(kafka_version): - self.skipTest("unsupported kafka version") + if not f(version): + pytest.skip("unsupported kafka version") - return func(self) + return func(*args, **kwargs) return wrapper - return kafka_versions + + return real_kafka_versions def get_open_port(): sock = socket.socket() @@ -80,6 +72,40 @@ def get_open_port(): sock.close() return port +_MESSAGES = {} +def msg(message): + """Format, encode and deduplicate a message + """ + global _MESSAGES #pylint: disable=global-statement + if message not in _MESSAGES: + _MESSAGES[message] = '%s-%s' % (message, str(uuid.uuid4())) + + return _MESSAGES[message].encode('utf-8') + +def send_messages(client, topic, partition, messages): + """Send messages to a topic's partition + """ + messages = [create_message(msg(str(m))) for m in messages] + produce = ProduceRequestPayload(topic, partition, messages=messages) + resp, = client.send_produce_request([produce]) + assert resp.error == 0 + + return [x.value for x in messages] + +def current_offset(client, topic, partition, kafka_broker=None): + """Get the current offset of a topic's partition + """ + try: + offsets, = client.send_offset_request([OffsetRequestPayload(topic, + partition, -1, 1)]) + except Exception: + # XXX: We've seen some UnknownErrors here and can't debug w/o server logs + if kafka_broker: + kafka_broker.dump_logs() + raise + else: + return offsets.offsets[0] + class KafkaIntegrationTestCase(unittest.TestCase): create_client = True topic = None @@ -98,7 +124,30 @@ def setUp(self): if self.create_client: self.client = SimpleClient('%s:%d' % (self.server.host, self.server.port)) - self.client.ensure_topic_exists(self.topic) + timeout = time.time() + 30 + while time.time() < timeout: + try: + self.client.load_metadata_for_topics(self.topic, ignore_leadernotavailable=False) + if self.client.has_metadata_for_topic(topic): + break + except (LeaderNotAvailableError, InvalidTopicError): + time.sleep(1) + else: + raise KafkaTimeoutError('Timeout loading topic metadata!') + + + # Ensure topic partitions have been created on all brokers to avoid UnknownPartitionErrors + # TODO: It might be a good idea to move this to self.client.ensure_topic_exists + for partition in self.client.get_partition_ids_for_topic(self.topic): + while True: + try: + req = OffsetRequestPayload(self.topic, partition, -1, 100) + self.client.send_offset_request([req]) + break + except (NotLeaderForPartitionError, UnknownTopicOrPartitionError, FailedPayloadsError) as e: + if time.time() > timeout: + raise KafkaTimeoutError('Timeout loading topic metadata!') + time.sleep(.1) self._messages = {} @@ -112,7 +161,8 @@ def tearDown(self): def current_offset(self, topic, partition): try: - offsets, = self.client.send_offset_request([OffsetRequestPayload(topic, partition, -1, 1)]) + offsets, = self.client.send_offset_request([OffsetRequestPayload(topic, + partition, -1, 1)]) except Exception: # XXX: We've seen some UnknownErrors here and can't debug w/o server logs self.zk.child.dump_logs() @@ -122,7 +172,7 @@ def current_offset(self, topic, partition): return offsets.offsets[0] def msgs(self, iterable): - return [ self.msg(x) for x in iterable ] + return [self.msg(x) for x in iterable] def msg(self, s): if s not in self._messages: diff --git a/tox.ini b/tox.ini index c2558f51e..ad95f9374 100644 --- a/tox.ini +++ b/tox.ini @@ -11,16 +11,16 @@ log_format = %(created)f %(filename)-23s %(threadName)s %(message)s deps = pytest pytest-cov - pytest-catchlog - py{27,34,35,36,py}: pylint==1.8.0 + py{27,34,35,36,py}: pylint==1.8.2 py{27,34,35,36,py}: pytest-pylint - pytest-sugar pytest-mock mock python-snappy lz4 xxhash + crc32c py26: unittest2 + decorator commands = py.test {posargs:--pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka --cov-config=.covrc} setenv =