summaryrefslogtreecommitdiff
path: root/kafka/coordinator
Commit message (Collapse)AuthorAgeFilesLines
* Don't use `kafka.common` internally1.3.5Jeff Widman2018-06-052-2/+2
| | | | This finishes the split from `kafka.common` to `kafka.errors`/`kafka.structs`.
* Always acquire client lock before coordinator lock to avoid deadlocks (#1464)Dana Powers2018-04-181-59/+64
|
* Heartbeat thread start / closeDana Powers2018-03-231-2/+4
|
* Change levels for some heartbeat thread loggingDana Powers2018-03-231-3/+3
|
* Check for immediate failure when looking up coordinator in heartbeat thread ↵Dana Powers2018-03-231-1/+5
| | | | (#1457)
* KAFKA-5512; Awake the heartbeat thread when timetoNextHeartbeat is equal to ↵Dana Powers2018-03-101-0/+4
| | | | 0 (#1439)
* Remove old CommitFailed error message from coordinator (#1436)Dana Powers2018-03-101-11/+1
|
* Fix consumer poll stuck error when no available partition (#1375)Yu Kou2018-02-081-1/+1
|
* use absolute imports everywhere (#1362)Kevin Tindall2018-02-064-23/+23
|
* KAFKA-3949: Avoid race condition when subscription changes during rebalance ↵Dana Powers2018-02-022-50/+76
| | | | (#1364)
* Name heartbeat thread with group_id; use backoff when polling (#1345)Dana Powers2018-01-161-1/+1
|
* Fix coordinator join_future race condition (#1338)Dana Powers2018-01-111-4/+8
| | | * Fix race condition in coordinator join_future handling
* Improve KafkaConsumer cleanup (#1339)Dana Powers2018-01-102-11/+28
|
* KAFKA-3888 Use background thread to process consumer heartbeats (#1266)Dana Powers2017-12-213-350/+615
|
* Replace periods (java) with underscores (python)Jeff Widman2017-11-071-2/+2
|
* Handle lookup_coordinator send failures (#1279)Dana Powers2017-10-241-5/+11
|
* KAFKA-4034: Avoid unnecessary consumer coordinator lookup (#1254)Dana Powers2017-10-112-8/+43
|
* Explicitly check for `None` rather than FalseJeff Widman2017-10-061-1/+1
| | | | | If the group leader somehow gets in a state that it has an empty partition assignment, then `self._assignment_snapshot` will be `{}` which evaluates to `False`. So `self._subscription.mark_for_reassignment()` will never be triggered, even if `self._assignment_snapshot != self._metadata_snapshot`. Fixes the symptoms of https://github.com/dpkp/kafka-python/issues/1237 although I suspect there's an additional bug in that case that triggers the condition of the the group leader getting an empty partition assignment.
* Initialize metadata_snapshot in group coordinator (#1174)Dana Powers2017-08-131-6/+9
|
* Use for join-time-max and sync-time-max metrics Max() measure function (#1146)Alexey Pervushin2017-07-201-2/+2
|
* change_subscription called only when necessary (#1132)Petr Šebek2017-07-071-2/+3
| | | | | | When we are using subscription by pattern change subscription is called every metadata update even when nothing changes. This PR ensures that change subscription is called only when set of topics changes.
* Backoff on unavailable group coordinator retry (#1125)Dana Powers2017-06-191-0/+2
|
* Additional docstrings for autocommit close optionDana Powers2017-03-132-2/+10
|
* Optionally skip auto-commit during consumer.close (#1031)Dana Powers2017-03-131-2/+3
|
* Short-circuit group coordinator requests when NodeNotReady (#995)Dana Powers2017-03-091-0/+23
|
* Avoid unknown coordinator after client poll (#1023)Dana Powers2017-03-091-7/+6
|
* For 0.8.2, only attempt connection to coordinator if least_loaded_node succeedsDana Powers2017-03-071-1/+2
|
* Minor additional logging for consumer coordinatorDana Powers2017-03-061-0/+2
|
* Dont refresh metadata on failed group coordinator request unless needed (#1006)Dana Powers2017-03-031-6/+4
|
* Add sphinx formatting to hyperlink methods (#898)Jeff Widman2017-03-031-5/+5
|
* PEP-8: Spacing & removed unused imports (#899)Jeff Widman2017-02-091-6/+6
|
* Fix possible request draining in ensure_active_group (#896)Dana Powers2016-12-021-3/+6
|
* Always include an error for logging when the coordinator is marked dead (#890)Dana Powers2016-11-182-9/+9
|
* Add comment for round robin partitioner with different subscriptionsDana Powers2016-08-201-0/+15
|
* Drain pending requests to the coordinator before initiating group rejoin (#798)Dana Powers2016-08-041-0/+8
|
* Change coordinator.needs_join_prepare -> coordinator.rejoining; fix consumer ↵Dana Powers2016-08-041-4/+4
| | | | group test (#795)
* Treat metric_group_prefix as config in KafkaConsumerDana Powers2016-08-042-8/+7
|
* Always absolute_importabsolute_importDana Powers2016-08-024-0/+8
|
* Vendor six 1.10.0sixDana Powers2016-08-014-4/+4
|
* Add base coordinator metricsDana Powers2016-07-173-76/+80
|
* KAFKA-3486: fix autocommit when partitions assigned manually (#767 / #626)Dana Powers2016-07-171-44/+8
|
* KAFKA-3117: handle metadata updates during consumer rebalance (#766 / #701)Dana Powers2016-07-171-10/+24
|
* KAFKA-2832: Add a consumer config option to exclude internal topics (#765)Dana Powers2016-07-171-4/+8
| | | | Use exclude_internal_topics config in KafkaConsumer to avoid subscribe patterns matching internal topics Raise error during rebalance if subscribed topics are not authorized
* Fix KafkaConsumer autocommit for 0.8 brokers (#756 / #706)Dana Powers2016-07-162-20/+25
| | | | * Dont wait for group join to enable AutoCommitTask if broker version < 0.9 * For zookeeper offset storage, set a "coordinator" with least_loaded_node
* Avoid some exceptions in Coordinator.__del__ (#668)Dana Powers2016-04-252-2/+4
|
* Changing some commented out metrics to avoid future gotchas.Zack Dever2016-04-131-3/+3
|
* Beginnings of metrics instrumentation in kafka consumer.Zack Dever2016-04-131-41/+30
| | | | | | This adds the parent metrics instance to kafka consumer, which will eventually be used to instrument everything under consumer. To start I ported the java consumer coordinator metrics.
* Default sleep=True in client.pollDana Powers2016-04-081-1/+1
|
* Can no longer have coordinator_id if coordinator_unknown()Dana Powers2016-04-081-6/+0
|
* Use version-indexed lists for request/response protocol structsprotocol_versionsDana Powers2016-04-052-24/+21
|