Skip to content

Take latest master from main line #1

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 162 commits into from
Nov 2, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
162 commits
Select commit Hold shift + click to select a range
fd3eb91
Drop dependency on sphinxcontrib-napoleon
stanislavlevin Feb 17, 2019
940b1c5
Make NotEnoughReplicasError/NotEnoughReplicasAfterAppendError retriab…
le-linh Mar 3, 2019
37699be
Use test.fixtures.version not test.conftest.version to avoid warnings…
dpkp Mar 7, 2019
7a99013
Do not require client lock for read-only operations (#1730)
dpkp Mar 7, 2019
8c07925
Do network connections and writes in KafkaClient.poll() (#1729)
dpkp Mar 8, 2019
2a91ca1
Synchronize puts to KafkaConsumer protocol buffer during async sends
dpkp Mar 12, 2019
994d283
Ignore lookup_coordinator result in commit_offsets_async (#1712)
asafflesch Mar 13, 2019
c0add71
Update travis test coverage: 2.7, 3.4, 3.7, pypy2.7 (#1614)
jeffwidman Mar 13, 2019
1904b53
Catch thrown OSError by python 3.7 when creating a connection (#1694)
danjo133 Mar 13, 2019
7965460
1701 use last offset from fetch v4 if available (#1724)
keithks Mar 13, 2019
921c553
Attempt to join heartbeat thread during close() (#1735)
dpkp Mar 14, 2019
1cd505d
Don't recheck version if api_versions data is already cached (#1738)
dpkp Mar 14, 2019
302b30c
Recheck connecting nodes sooner when refreshing metadata (#1737)
dpkp Mar 14, 2019
39dd8c2
Mock dns lookups in test_conn (#1739)
dpkp Mar 14, 2019
5d7a5fc
Minor updates to client_async.py
dpkp Mar 14, 2019
703f065
Fix default protocol parser version
dpkp Mar 14, 2019
812de35
Retry bootstrapping after backoff when necessary (#1736)
dpkp Mar 14, 2019
965d21b
Error if connections_max_idle_ms not larger than request_timeout_ms (…
jeffwidman Mar 15, 2019
2257419
Release 1.4.5
dpkp Mar 15, 2019
7e045f2
Bump version for development
dpkp Mar 15, 2019
2983264
Forgot compatibility section in docs/changelog.rst
dpkp Mar 15, 2019
ee4a53e
Fix race condition in protocol.send_bytes (#1752)
isamaru Mar 21, 2019
64f70b5
Generate SSL certificates for local testing (#1756)
dpkp Mar 21, 2019
f2f2bfe
Wrap SSL sockets after connecting (#1754)
dpkp Mar 22, 2019
0bc7518
Allow configuration of SSL Ciphers (#1755)
dpkp Mar 22, 2019
af2dd48
Maintain shadow cluster metadata for bootstrapping (#1753)
dpkp Mar 22, 2019
d620703
Add py to requirements-dev
dpkp Mar 22, 2019
d032844
Fix flaky conn tests that use time.time (#1758)
dpkp Mar 22, 2019
8e2ed3e
Support SASL OAuthBearer Authentication (#1750)
pt2pham Mar 23, 2019
ce9c1d2
Update sasl configuration docstrings
dpkp Mar 24, 2019
d388b48
Dont do client wakeup when sending from sender thread (#1761)
dpkp Mar 24, 2019
8602389
Rename ssl.keystore.location and ssl.truststore.location config files
dpkp Mar 22, 2019
e45b89b
Send pending requests before waiting for responses (#1762)
dpkp Mar 27, 2019
227a946
Revert 703f0659 / fix 0.8.2 protocol quick detection (#1763)
dpkp Mar 27, 2019
de6e9d3
Fixups to benchmark scripts for py3 / new KafkaFixture interface
dpkp Mar 30, 2019
b1effa2
Dont wakeup during maybe_refresh_metadata -- it is only called by pol…
dpkp Mar 31, 2019
3664ae8
lock client.check_version (#1771)
dpkp Apr 1, 2019
c02df4b
Avoid race condition on client._conns in send() (#1772)
dpkp Apr 1, 2019
51313d7
Dont treat popped conn.close() as failure in state change callback (#…
dpkp Apr 2, 2019
827dee2
Fix possible AttribueError during conn._close_socket (#1776)
dpkp Apr 2, 2019
6ca6f36
Reset reconnect backoff on SSL connection (#1777)
dpkp Apr 2, 2019
ed4cab6
Return connection state explicitly after close in connect() (#1778)
dpkp Apr 2, 2019
27cd93b
Additional BrokerConnection locks to synchronize protocol/IFR state (…
dpkp Apr 2, 2019
91d3149
Do not call state_change_callback with lock (#1775)
dpkp Apr 3, 2019
be7f935
Release 1.4.6
dpkp Apr 3, 2019
f6a8a38
Bump version for development of next release
dpkp Apr 3, 2019
f854791
A little python cleanup (#1805)
jeffwidman May 17, 2019
1b6c9c7
Update link to kafka docs
jeffwidman May 17, 2019
f145e37
Use futures to parallelize calls to _send_request_to_node() (#1807)
davidheitman May 21, 2019
edfafc0
Remove unused imports (#1808)
jeffwidman May 23, 2019
1f73287
Make partitions_for_topic a read-through cache (#1781)
Baisang May 23, 2019
1a0f297
Fix typo in _fetch_all_topic_metadata function (#1809)
Baisang May 23, 2019
cee4d17
Update docs for api_version_auto_timeout_ms (#1812)
jeffwidman May 24, 2019
21b00c3
Remove unused/weird comment line (#1813)
jeffwidman May 29, 2019
5bb1abd
Catch TimeoutError in BrokerConnection send/recv (#1820)
dpkp May 30, 2019
9f0b518
Reduce client poll timeout when no ifrs
dpkp May 29, 2019
79dd508
Delay converting to seconds
jeffwidman May 30, 2019
f126e5b
Sanity test for consumer.topics() and consumer.partitions_for_topic()
Baisang Jun 1, 2019
91f4642
Use dedicated connection for group coordinator (#1822)
dpkp Jun 19, 2019
01053da
Break consumer operations into request / response methods (#1845)
jeffwidman Jun 20, 2019
5e055bc
Allow the coordinator to auto-commit for all api_version.
justecorruptio Jun 3, 2019
5a72a62
Update KafkaAdminClient Docs
cardy31 Jun 21, 2019
279a7dd
Add the `sasl_kerberos_domain_name` arg to `KafkaAdminClient`
jeffwidman Jun 28, 2019
fb87a35
Update link to upstream Kafka docs
jeffwidman Jul 11, 2019
eed25fc
Fix minor typo (#1865)
carsonip Jul 14, 2019
ea35fdf
Break FindCoordinator into request/response methods
jeffwidman Jul 31, 2019
2180d31
tests: Use socket.SOCK_STREAM in assertions
iv-m Aug 12, 2019
ace6af5
Update conn.py
iAnomaly Aug 15, 2019
5bc2529
Reduce internal client poll timeout for consumer iterator interface (…
dpkp Aug 16, 2019
e49caeb
Minor test cleanup (#1885)
jeffwidman Aug 22, 2019
98c0058
Cleanup handling of KAFKA_VERSION env var in tests (#1887)
jeffwidman Aug 22, 2019
6e6d0cc
Bump integration tests to 0.10.2.2 and 0.11.0.3 (#1890)
jeffwidman Aug 23, 2019
61fa0b2
Convert remaining `KafkaConsumer` tests to `pytest` (#1886)
jeffwidman Aug 23, 2019
7a69952
Improve connection lock handling; always use context manager (#1895)
dpkp Sep 3, 2019
5e4d151
kafka-python: Fixed crc32c avilability on non-intel architectures. (#…
ossdev07 Sep 25, 2019
76ad662
Add ACL api to KafkaAdminClient (#1833)
ulrikjohansson Sep 28, 2019
5381591
Fixup test_admin_integration test fixtures
dpkp Sep 28, 2019
98ebff8
Fix Admin Client api version checking; only test ACL integration on 0…
dpkp Sep 29, 2019
580fc0c
Update kafka.properties for ACL tests (0.11+ only)
dpkp Sep 29, 2019
a9f513c
Skip admin integration tests -- travis fixture issues
dpkp Sep 29, 2019
5d1d424
Wrap consumer.poll() for KafkaConsumer iteration (#1902)
dpkp Sep 29, 2019
89bf6a6
Rely on socket selector to detect completed connection attempts (#1909)
dpkp Sep 29, 2019
9de12d3
Do not use wakeup when sending fetch requests from consumer (#1911)
dpkp Sep 29, 2019
392d674
Send socket data via non-blocking IO with send buffer (#1912)
dpkp Sep 30, 2019
0f929bd
Change coordinator lock acquisition order (#1821)
dpkp Sep 30, 2019
298cb0d
Issue #1780 - Consumer hang indefinitely in fetcher._retrieve_offsets…
royantman Sep 30, 2019
7a7a890
Added a function to determine if bootstrap is successfully connected …
Wayde2014 Sep 30, 2019
87fb1bb
Improve/refactor bootstrap_connected
dpkp Sep 30, 2019
975087b
Follow up to PR 1782 -- fix tests (#1914)
dpkp Sep 30, 2019
0a8884b
Add KAFKA 2.3.0 to test matrix (#1915)
dpkp Sep 30, 2019
0552b04
Release 1.4.7 (#1916)
dpkp Sep 30, 2019
f1cda98
Update docstring to match conn.py's (#1921)
Oct 3, 2019
84e37e0
convert test_admin_integration to pytest (#1923)
ulrikjohansson Oct 7, 2019
6d3800c
Fix describe config for multi-broker clusters (#1869)
jlandersen Oct 11, 2019
3631bfa
Remove SimpleClient, Producer, Consumer, Unittest (#1196)
jeffwidman Oct 11, 2019
f61b08c
Remove deprecated `ConnectionError` (#1816)
jeffwidman Oct 11, 2019
cb7fd01
Remove unused/empty .gitsubmodules file (#1928)
jeffwidman Oct 11, 2019
736218d
Fix doc import paths (#1933)
jeffwidman Oct 21, 2019
3861e16
Fix typos
carsonip Oct 25, 2019
2ed01e9
Fix simple typo: managementment -> management
timgates42 Dec 7, 2019
b59323d
xfail test_describe_configs_topic_resource_returns_configs (Issue #1929)
dpkp Dec 29, 2019
cf28da8
Improve docs for reconnect_backoff_max_ms (#1976)
dpkp Dec 29, 2019
31f846c
Add crc32c to README and docs
dpkp Dec 29, 2019
ee1c4a4
Enable SCRAM-SHA-256 and SCRAM-SHA-512 for sasl (#1918)
swenzel Dec 29, 2019
e3362ac
Style updates to scram sasl support
dpkp Dec 29, 2019
e06ea70
Admin protocol updates (#1948)
TylerLubeck Dec 29, 2019
875dd85
Fix typo
dpkp Dec 29, 2019
5c477f2
Fixes KafkaAdminClient returning `IncompatibleBrokerVersion` when pas…
ian28223 Dec 29, 2019
bc25877
Implement __eq__ and __hash__ for ACL objects (#1955)
TylerLubeck Dec 29, 2019
41d9f1c
Log retriable coordinator NodeNotReady, TooManyInFlightRequests as de…
dpkp Dec 29, 2019
3aada77
Reset conn configs on exception in conn.check_version() (#1977)
dpkp Dec 29, 2019
23534b4
Raise AssertionError if consumer closed in poll() (#1978)
dpkp Dec 30, 2019
1a91a54
Do not block on sender thread join after timeout in producer.close() …
dpkp Dec 30, 2019
2a86b23
Optionally return OffsetAndMetadata from consumer.committed(tp) (#1979)
dpkp Dec 30, 2019
46882a8
Do not log topic-specific errors in full metadata fetch (#1980)
dpkp Dec 30, 2019
1bb6b72
kafka 2.4.0 (#1972)
vvuibert Dec 30, 2019
bb1c13e
Update 2.4.0 resource files for sasl integration
dpkp Dec 30, 2019
d54aaf6
Fix slots usage and use more slots
carsonip Jan 17, 2020
385f603
Fix benchmarks to use pyperf
carsonip Jan 17, 2020
7008fd4
Use ==/!= to compare str, bytes, and int literals
cclauss Jan 10, 2020
66f9750
Use print() function in both Python 2 and Python 3
cclauss Jan 10, 2020
6babefa
Remove some dead code
jeffwidman Aug 30, 2017
3d98741
Remove unused import
jeffwidman Feb 6, 2020
209515b
Implement methods to convert a Struct object to a pythonic object (#1…
TylerLubeck Feb 6, 2020
da01fef
Implement list_topics, describe_topics, and describe_cluster
Nov 15, 2019
f92889a
Use the controller for topic metadata requests
TylerLubeck Feb 6, 2020
6e2978e
_send_request_to_controller returns a raw result, not a future
TylerLubeck Feb 6, 2020
ed8c9cb
Release 2.0.0
dpkp Feb 11, 2020
bbb8c90
Bump version for development of next release
dpkp Feb 11, 2020
7195f03
Fix topic error parsing in MetadataResponse
Feb 11, 2020
3c3fdc1
KAFKA-8962: Use least_loaded_node() for describe_topics()
jeffwidman Feb 16, 2020
fcc9a30
Patch Release 2.0.1
dpkp Feb 20, 2020
615b9e0
Bump version for development
dpkp Feb 20, 2020
d1dfb6d
Set length of header value to 0 if None
kvfi Feb 24, 2020
34dc36d
Fix docs by adding SASL mention
jeffwidman Mar 2, 2020
5d4b3ec
Add 2.1.1 to build_integration (#2019)
Mar 16, 2020
f9e0264
Add `log_start_offset` to message protocol parsing (#2020)
Mar 25, 2020
6fc0081
Add logic for inferring newer broker versions (#2038)
May 5, 2020
f5a99fa
Rename README to README.md (#2055)
qhzxc0015 May 26, 2020
316da74
remove unused imports
jameslamb May 11, 2020
bd557da
Python 3.8 support (#2088)
Photonios Jul 23, 2020
91daea3
Fix #1985: fix consumer deadlock when heartbeat thread request timeou…
huangcuiyang Sep 7, 2020
8e7ea69
Fix typo (#2096)
haracejacob Sep 7, 2020
08ea211
fixed the broken compatibility page link (#2045)
anuragrana Sep 7, 2020
a27ab88
Add support for `zstd` compression (#2021)
Sep 7, 2020
f6677cf
Add kafka.structs docstrings (#2080)
mmenbawy Sep 7, 2020
3a9d830
Update example.py (#2081)
mmenbawy Sep 7, 2020
e4913db
producer/kafka: Disable logging during object destruction (#2043)
gioele Sep 7, 2020
26b8400
Enhancement for Kafka Admin Client's "Describe Consumer Group" (#2035)
Apurva007 Sep 17, 2020
d09ff02
Added crc32c extra requirement as per issue #2068
Misharmis Jun 23, 2020
b32f369
Allow configurable timeouts in admin client check version
sunnyakaxd Aug 22, 2020
e485a6e
Fix initialization order in KafkaClient (#2119)
pecalleja Sep 17, 2020
16a0b31
Feature: delete consumergroups (#2040)
swenzel Sep 17, 2020
6cfe706
Lint cleanup (#2126)
jeffwidman Sep 17, 2020
098ecbf
Merge _find_coordinator_id methods (#2127)
jeffwidman Sep 17, 2020
16f4867
Fix crc32c deprecation warning (#2128)
jeffwidman Sep 17, 2020
cb96a1a
Bump dev requirements (#2129)
jeffwidman Sep 17, 2020
c536dd2
KIP-54: Implement sticky partition assignment strategy (#2057)
aynroot Sep 30, 2020
5bb126b
Patch Release 2.0.2
dpkp Sep 30, 2020
eb070d6
Add kafka-2.5 to compatibility
dpkp Sep 30, 2020
6f932ba
Bump version for development
dpkp Sep 30, 2020
83b7b27
Use six.viewitems instead of six.iteritems to avoid encoding problems…
aynroot Oct 31, 2020
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 14 additions & 7 deletions .travis.yml
Original file line number Diff line number Diff line change
@@ -1,38 +1,45 @@
language: python

dist: xenial

python:
- 2.7
- 3.4
- 3.5
- 3.6
- pypy
- 3.7
- 3.8
- pypy2.7-6.0

env:
- KAFKA_VERSION=0.8.2.2
- KAFKA_VERSION=0.9.0.1
- KAFKA_VERSION=0.10.2.1
- KAFKA_VERSION=0.11.0.2
- KAFKA_VERSION=0.10.2.2
- KAFKA_VERSION=0.11.0.3
- KAFKA_VERSION=1.1.1
- KAFKA_VERSION=2.4.0
- KAFKA_VERSION=2.5.0

addons:
apt:
packages:
- libsnappy-dev
- libzstd-dev
- openjdk-8-jdk

cache:
directories:
- $HOME/.cache/pip
- servers/
- servers/dist

before_install:
- source travis_java_install.sh
- ./build_integration.sh

install:
- pip install tox coveralls
- pip install .

script:
- tox -e `if [ "$TRAVIS_PYTHON_VERSION" == "pypy" ]; then echo pypy; else echo py${TRAVIS_PYTHON_VERSION/./}; fi`
- tox -e `if [ "$TRAVIS_PYTHON_VERSION" == "pypy2.7-6.0" ]; then echo pypy; else echo py${TRAVIS_PYTHON_VERSION/./}; fi`

after_success:
- coveralls
241 changes: 241 additions & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
@@ -1,3 +1,244 @@
# 2.0.2 (Sep 29, 2020)

Consumer
* KIP-54: Implement sticky partition assignment strategy (aynroot / PR #2057)
* Fix consumer deadlock when heartbeat thread request timeout (huangcuiyang / PR #2064)

Compatibility
* Python 3.8 support (Photonios / PR #2088)

Cleanups
* Bump dev requirements (jeffwidman / PR #2129)
* Fix crc32c deprecation warning (crc32c==2.1) (jeffwidman / PR #2128)
* Lint cleanup (jeffwidman / PR #2126)
* Fix initialization order in KafkaClient (pecalleja / PR #2119)
* Allow installing crc32c via extras (mishas / PR #2069)
* Remove unused imports (jameslamb / PR #2046)

Admin Client
* Merge _find_coordinator_id methods (jeffwidman / PR #2127)
* Feature: delete consumergroups (swenzel / PR #2040)
* Allow configurable timeouts in admin client check version (sunnyakaxd / PR #2107)
* Enhancement for Kafka Admin Client's "Describe Consumer Group" (Apurva007 / PR #2035)

Protocol
* Add support for zstd compression (gabriel-tincu / PR #2021)
* Add protocol support for brokers 1.1.0 - 2.5.0 (gabriel-tincu / PR #2038)
* Add ProduceRequest/ProduceResponse v6/v7/v8 (gabriel-tincu / PR #2020)
* Fix parsing NULL header values (kvfi / PR #2024)

Tests
* Add 2.5.0 to automated CI tests (gabriel-tincu / PR #2038)
* Add 2.1.1 to build_integration (gabriel-tincu / PR #2019)

Documentation / Logging / Errors
* Disable logging during producer object gc (gioele / PR #2043)
* Update example.py; use threading instead of multiprocessing (Mostafa-Elmenbawy / PR #2081)
* Fix typo in exception message (haracejacob / PR #2096)
* Add kafka.structs docstrings (Mostafa-Elmenbawy / PR #2080)
* Fix broken compatibility page link (anuragrana / PR #2045)
* Rename README to README.md (qhzxc0015 / PR #2055)
* Fix docs by adding SASL mention (jeffwidman / #1990)

# 2.0.1 (Feb 19, 2020)

Admin Client
* KAFKA-8962: Use least_loaded_node() for AdminClient.describe_topics() (jeffwidman / PR #2000)
* Fix AdminClient topic error parsing in MetadataResponse (jtribble / PR #1997)

# 2.0.0 (Feb 10, 2020)

This release includes breaking changes for any application code that has not
migrated from older Simple-style classes to newer Kafka-style classes.

Deprecation
* Remove deprecated SimpleClient, Producer, Consumer, Unittest (jeffwidman / PR #1196)

Admin Client
* Use the controller for topic metadata requests (TylerLubeck / PR #1995)
* Implement list_topics, describe_topics, and describe_cluster (TylerLubeck / PR #1993)
* Implement __eq__ and __hash__ for ACL objects (TylerLubeck / PR #1955)
* Fixes KafkaAdminClient returning `IncompatibleBrokerVersion` when passing an `api_version` (ian28223 / PR #1953)
* Admin protocol updates (TylerLubeck / PR #1948)
* Fix describe config for multi-broker clusters (jlandersen / PR #1869)

Miscellaneous Bugfixes / Improvements
* Enable SCRAM-SHA-256 and SCRAM-SHA-512 for sasl (swenzel / PR #1918)
* Fix slots usage and use more slots (carsonip / PR #1987)
* Optionally return OffsetAndMetadata from consumer.committed(tp) (dpkp / PR #1979)
* Reset conn configs on exception in conn.check_version() (dpkp / PR #1977)
* Do not block on sender thread join after timeout in producer.close() (dpkp / PR #1974)
* Implement methods to convert a Struct object to a pythonic object (TylerLubeck / PR #1951)

Test Infrastructure / Documentation / Maintenance
* Update 2.4.0 resource files for sasl integration (dpkp)
* Add kafka 2.4.0 to CI testing (vvuibert / PR #1972)
* convert test_admin_integration to pytest (ulrikjohansson / PR #1923)
* xfail test_describe_configs_topic_resource_returns_configs (dpkp / Issue #1929)
* Add crc32c to README and docs (dpkp)
* Improve docs for reconnect_backoff_max_ms (dpkp / PR #1976)
* Fix simple typo: managementment -> management (timgates42 / PR #1966)
* Fix typos (carsonip / PR #1938)
* Fix doc import paths (jeffwidman / PR #1933)
* Update docstring to match conn.py's (dabcoder / PR #1921)
* Do not log topic-specific errors in full metadata fetch (dpkp / PR #1980)
* Raise AssertionError if consumer closed in poll() (dpkp / PR #1978)
* Log retriable coordinator NodeNotReady, TooManyInFlightRequests as debug not error (dpkp / PR #1975)
* Remove unused import (jeffwidman)
* Remove some dead code (jeffwidman)
* Fix a benchmark to Use print() function in both Python 2 and Python 3 (cclauss / PR #1983)
* Fix a test to use ==/!= to compare str, bytes, and int literals (cclauss / PR #1984)
* Fix benchmarks to use pyperf (carsonip / PR #1986)
* Remove unused/empty .gitsubmodules file (jeffwidman / PR #1928)
* Remove deprecated `ConnectionError` (jeffwidman / PR #1816)


# 1.4.7 (Sep 30, 2019)

This is a minor release focused on KafkaConsumer performance, Admin Client
improvements, and Client concurrency. The KafkaConsumer iterator implementation
has been greatly simplified so that it just wraps consumer.poll(). The prior
implementation will remain available for a few more releases using the optional
KafkaConsumer config: `legacy_iterator=True` . This is expected to improve
consumer throughput substantially and help reduce heartbeat failures / group
rebalancing.

Client
* Send socket data via non-blocking IO with send buffer (dpkp / PR #1912)
* Rely on socket selector to detect completed connection attempts (dpkp / PR #1909)
* Improve connection lock handling; always use context manager (melor,dpkp / PR #1895)
* Reduce client poll timeout when there are no in-flight requests (dpkp / PR #1823)

KafkaConsumer
* Do not use wakeup when sending fetch requests from consumer (dpkp / PR #1911)
* Wrap `consumer.poll()` for KafkaConsumer iteration (dpkp / PR #1902)
* Allow the coordinator to auto-commit on old brokers (justecorruptio / PR #1832)
* Reduce internal client poll timeout for (legacy) consumer iterator interface (dpkp / PR #1824)
* Use dedicated connection for group coordinator (dpkp / PR #1822)
* Change coordinator lock acquisition order (dpkp / PR #1821)
* Make `partitions_for_topic` a read-through cache (Baisang / PR #1781,#1809)
* Fix consumer hanging indefinitely on topic deletion while rebalancing (commanderdishwasher / PR #1782)

Miscellaneous Bugfixes / Improvements
* Fix crc32c avilability on non-intel architectures (ossdev07 / PR #1904)
* Load system default SSL CAs if `ssl_cafile` is not provided (iAnomaly / PR #1883)
* Catch py3 TimeoutError in BrokerConnection send/recv (dpkp / PR #1820)
* Added a function to determine if bootstrap is successfully connected (Wayde2014 / PR #1876)

Admin Client
* Add ACL api support to KafkaAdminClient (ulrikjohansson / PR #1833)
* Add `sasl_kerberos_domain_name` config to KafkaAdminClient (jeffwidman / PR #1852)
* Update `security_protocol` config documentation for KafkaAdminClient (cardy31 / PR #1849)
* Break FindCoordinator into request/response methods in KafkaAdminClient (jeffwidman / PR #1871)
* Break consumer operations into request / response methods in KafkaAdminClient (jeffwidman / PR #1845)
* Parallelize calls to `_send_request_to_node()` in KafkaAdminClient (davidheitman / PR #1807)

Test Infrastructure / Documentation / Maintenance
* Add Kafka 2.3.0 to test matrix and compatibility docs (dpkp / PR #1915)
* Convert remaining `KafkaConsumer` tests to `pytest` (jeffwidman / PR #1886)
* Bump integration tests to 0.10.2.2 and 0.11.0.3 (jeffwidman / #1890)
* Cleanup handling of `KAFKA_VERSION` env var in tests (jeffwidman / PR #1887)
* Minor test cleanup (jeffwidman / PR #1885)
* Use `socket.SOCK_STREAM` in test assertions (iv-m / PR #1879)
* Sanity test for `consumer.topics()` and `consumer.partitions_for_topic()` (Baisang / PR #1829)
* Cleanup seconds conversion in client poll timeout calculation (jeffwidman / PR #1825)
* Remove unused imports (jeffwidman / PR #1808)
* Cleanup python nits in RangePartitionAssignor (jeffwidman / PR #1805)
* Update links to kafka consumer config docs (jeffwidman)
* Fix minor documentation typos (carsonip / PR #1865)
* Remove unused/weird comment line (jeffwidman / PR #1813)
* Update docs for `api_version_auto_timeout_ms` (jeffwidman / PR #1812)


# 1.4.6 (Apr 2, 2019)

This is a patch release primarily focused on bugs related to concurrency,
SSL connections and testing, and SASL authentication:

Client Concurrency Issues (Race Conditions / Deadlocks)
* Fix race condition in `protocol.send_bytes` (isamaru / PR #1752)
* Do not call `state_change_callback` with lock (dpkp / PR #1775)
* Additional BrokerConnection locks to synchronize protocol/IFR state (dpkp / PR #1768)
* Send pending requests before waiting for responses (dpkp / PR #1762)
* Avoid race condition on `client._conns` in send() (dpkp / PR #1772)
* Hold lock during `client.check_version` (dpkp / PR #1771)

Producer Wakeup / TimeoutError
* Dont wakeup during `maybe_refresh_metadata` -- it is only called by poll() (dpkp / PR #1769)
* Dont do client wakeup when sending from sender thread (dpkp / PR #1761)

SSL - Python3.7 Support / Bootstrap Hostname Verification / Testing
* Wrap SSL sockets after connecting for python3.7 compatibility (dpkp / PR #1754)
* Allow configuration of SSL Ciphers (dpkp / PR #1755)
* Maintain shadow cluster metadata for bootstrapping (dpkp / PR #1753)
* Generate SSL certificates for local testing (dpkp / PR #1756)
* Rename ssl.keystore.location and ssl.truststore.location config files (dpkp)
* Reset reconnect backoff on SSL connection (dpkp / PR #1777)

SASL - OAuthBearer support / api version bugfix
* Fix 0.8.2 protocol quick detection / fix SASL version check (dpkp / PR #1763)
* Update sasl configuration docstrings to include supported mechanisms (dpkp)
* Support SASL OAuthBearer Authentication (pt2pham / PR #1750)

Miscellaneous Bugfixes
* Dont force metadata refresh when closing unneeded bootstrap connections (dpkp / PR #1773)
* Fix possible AttributeError during conn._close_socket (dpkp / PR #1776)
* Return connection state explicitly after close in connect() (dpkp / PR #1778)
* Fix flaky conn tests that use time.time (dpkp / PR #1758)
* Add py to requirements-dev (dpkp)
* Fixups to benchmark scripts for py3 / new KafkaFixture interface (dpkp)


# 1.4.5 (Mar 14, 2019)

This release is primarily focused on addressing lock contention
and other coordination issues between the KafkaConsumer and the
background heartbeat thread that was introduced in the 1.4 release.

Consumer
* connections_max_idle_ms must be larger than request_timeout_ms (jeffwidman / PR #1688)
* Avoid race condition during close() / join heartbeat thread (dpkp / PR #1735)
* Use last offset from fetch v4 if available to avoid getting stuck in compacted topic (keithks / PR #1724)
* Synchronize puts to KafkaConsumer protocol buffer during async sends (dpkp / PR #1733)
* Improve KafkaConsumer join group / only enable Heartbeat Thread during stable group (dpkp / PR #1695)
* Remove unused `skip_double_compressed_messages` (jeffwidman / PR #1677)
* Fix commit_offsets_async() callback (Faqa / PR #1712)

Client
* Retry bootstrapping after backoff when necessary (dpkp / PR #1736)
* Recheck connecting nodes sooner when refreshing metadata (dpkp / PR #1737)
* Avoid probing broker versions twice on newer brokers (dpkp / PR #1738)
* Move all network connections and writes to KafkaClient.poll() (dpkp / PR #1729)
* Do not require client lock for read-only operations (dpkp / PR #1730)
* Timeout all unconnected conns (incl SSL) after request_timeout_ms (dpkp / PR #1696)

Admin Client
* Fix AttributeError in response topic error codes checking (jeffwidman)
* Fix response error checking in KafkaAdminClient send_to_controller (jeffwidman)
* Fix NotControllerError check (jeffwidman)

Core/Protocol
* Fix default protocol parser version / 0.8.2 version probe (dpkp / PR #1740)
* Make NotEnoughReplicasError/NotEnoughReplicasAfterAppendError retriable (le-linh / PR #1722)

Bugfixes
* Use copy() in metrics() to avoid thread safety issues (emeric254 / PR #1682)

Test Infrastructure
* Mock dns lookups in test_conn (dpkp / PR #1739)
* Use test.fixtures.version not test.conftest.version to avoid warnings (dpkp / PR #1731)
* Fix test_legacy_correct_metadata_response on x86 arch (stanislavlevin / PR #1718)
* Travis CI: 'sudo' tag is now deprecated in Travis (cclauss / PR #1698)
* Use Popen.communicate() instead of Popen.wait() (Baisang / PR #1689)

Compatibility
* Catch thrown OSError by python 3.7 when creating a connection (danjo133 / PR #1694)
* Update travis test coverage: 2.7, 3.4, 3.7, pypy2.7 (jeffwidman, dpkp / PR #1614)
* Drop dependency on sphinxcontrib-napoleon (stanislavlevin / PR #1715)
* Remove unused import from kafka/producer/record_accumulator.py (jeffwidman / PR #1705)
* Fix SSL connection testing in Python 3.7 (seanthegeek, silentben / PR #1669)


# 1.4.4 (Nov 20, 2018)

Bugfixes
Expand Down
6 changes: 3 additions & 3 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,8 @@ servers/$(KAFKA_VERSION)/kafka-bin:
build-integration: servers/$(KAFKA_VERSION)/kafka-bin

# Test and produce coverage using tox. This is the same as is run on Travis
test36: build-integration
KAFKA_VERSION=$(KAFKA_VERSION) SCALA_VERSION=$(SCALA_VERSION) tox -e py36 -- $(FLAGS)
test37: build-integration
KAFKA_VERSION=$(KAFKA_VERSION) SCALA_VERSION=$(SCALA_VERSION) tox -e py37 -- $(FLAGS)

test27: build-integration
KAFKA_VERSION=$(KAFKA_VERSION) SCALA_VERSION=$(SCALA_VERSION) tox -e py27 -- $(FLAGS)
Expand Down Expand Up @@ -56,4 +56,4 @@ doc:
make -C docs html
@echo "open file://`pwd`/docs/_build/html/index.html"

.PHONY: all test36 test27 test-local cov-local clean doc
.PHONY: all test37 test27 test-local cov-local clean doc
21 changes: 11 additions & 10 deletions README.rst
Original file line number Diff line number Diff line change
@@ -1,8 +1,8 @@
Kafka Python client
------------------------

.. image:: https://img.shields.io/badge/kafka-1.1%2C%201.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/badge/kafka-2.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg
:target: https://kafka-python.readthedocs.io/en/master/compatibility.html
.. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg
:target: https://pypi.python.org/pypi/kafka-python
.. image:: https://coveralls.io/repos/dpkp/kafka-python/badge.svg?branch=master&service=github
Expand Down Expand Up @@ -142,6 +142,14 @@ To enable snappy compression/decompression install python-snappy (also requires
See <https://kafka-python.readthedocs.io/en/master/install.html#optional-snappy-install>
for more information.

Optimized CRC32 Validation
**************************

Kafka uses CRC32 checksums to validate messages. kafka-python includes a pure
python implementation for compatibility. To improve performance for high-throughput
applications, kafka-python will use `crc32c` for optimized native code if installed.
See https://pypi.org/project/crc32c/

Protocol
********

Expand All @@ -150,11 +158,4 @@ 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 1.1+).

Low-level
*********

Legacy support is maintained for low-level consumer and producer classes,
SimpleConsumer and SimpleProducer. See
<https://kafka-python.readthedocs.io/en/master/simple.html?highlight=SimpleProducer> for API details.
(0.8.0 to 2.4+).
4 changes: 0 additions & 4 deletions benchmarks/README

This file was deleted.

4 changes: 4 additions & 0 deletions benchmarks/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
The `record_batch_*` benchmarks in this section are written using
``pyperf`` library, created by Victor Stinner. For more information on
how to get reliable results of test runs please consult
https://pyperf.readthedocs.io/en/latest/run_benchmark.html.
6 changes: 3 additions & 3 deletions benchmarks/consumer_performance.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ def start_brokers(n):
replicas = min(n, 3)
print('-> {0} Brokers [{1} partitions / {2} replicas]'.format(n, partitions, replicas))
brokers = [
KafkaFixture.instance(i, zk.host, zk.port, zk_chroot='',
KafkaFixture.instance(i, zk, zk_chroot='',
partitions=partitions, replicas=replicas)
for i in range(n)
]
Expand Down Expand Up @@ -148,7 +148,7 @@ def get_args_parser():
help='Topic for consumer test',
default='kafka-python-benchmark-test')
parser.add_argument(
'--num-records', type=long,
'--num-records', type=int,
help='number of messages to consume',
default=1000000)
parser.add_argument(
Expand All @@ -157,7 +157,7 @@ def get_args_parser():
default=100)
parser.add_argument(
'--consumer-config', type=str, nargs='+', default=(),
help='kafka consumer related configuaration properties like '
help='kafka consumer related configuration properties like '
'bootstrap_servers,client_id etc..')
parser.add_argument(
'--fixture-compression', type=str,
Expand Down
Loading