summaryrefslogtreecommitdiff
path: root/kafka/conn.py
Commit message (Collapse)AuthorAgeFilesLines
* Catch dns lookup errors when attempting to connect to nodesDana Powers2016-01-271-2/+8
|
* Warn on 0.8.2 GroupCoordinator no-topic-quirk, dont raise exceptionDana Powers2016-01-231-2/+5
|
* Pylint ignores to fix weird 2.7 error in new pylint versionDana Powers2016-01-231-0/+3
|
* Add DeprecationWarnings to legacy KafkaClient, Simple/MultiProcess/Consumer, ↵deprecation_warningsDana Powers2016-01-121-0/+3
| | | | and KafkaConnection
* Docstring updatesDana Powers2016-01-071-1/+10
|
* Catch py3 ConnectionErrorsDana Powers2016-01-031-7/+18
|
* Check for 0.8.2 GroupCoordinator quirk in BrokerConnectionDana Powers2016-01-031-1/+13
|
* Fix timeout bug in BrokerConnection.connect()Dana Powers2016-01-031-8/+9
|
* Fixup b6a2ad9: Fail with ConnectionErrors in BrokerConnection.sendDana Powers2016-01-021-2/+3
|
* Prefer assert or more-specific error to IllegalState / IllegalArgumentDana Powers2015-12-301-8/+3
|
* BrokerConnection: wrap socket errors in retriable ConnectionErrorDana Powers2015-12-301-3/+3
|
* Switch configs from attributes to dict to make passing / inspecting easierDana Powers2015-12-291-25/+31
|
* Use _next_correlation_id() method to avoid int overflowsDana Powers2015-12-291-4/+8
|
* Use ConnectionError rather than new DisconnectErrorDana Powers2015-12-291-2/+2
|
* Update BrokerConnection for use with async clientDana Powers2015-12-281-39/+74
| | | | | | | | | - use descriptive names for ConnectionStates enum values - Change default send_buffer_bytes config to 131072 - add can_send_more() and max_in_flight_requests_per_connection config - add blacked_out() and reconnect_backoff_ms config - last_attempt and last_failure are now public attributes - raise TooManyInFlightRequests in conn.send() if cant send more
* Switch BrokerConnection to (mostly) non-blocking IO.Dana Powers2015-12-171-81/+218
| | | | | | - return kafka.Future on send() - recv is now non-blocking call that completes futures when possible - update KafkaClient to block on future completion
* Check for no in-flight-requests in BrokerConnection.recvDana Powers2015-12-101-0/+3
|
* Move Request / Response logging from KafkaClient to BrokerConnectionDana Powers2015-12-101-2/+4
| | | | and reenable kafka.conn debug logging in tests
* Add next_correlation_id_recv + send helper methods to BrokerConnectionDana Powers2015-12-091-0/+8
|
* Support requests that do not expect a response in BrokerConnectionDana Powers2015-12-091-2/+3
|
* Exception handling cleanup in BrokerConnection (also catch struct.errors in ↵Dana Powers2015-12-091-4/+4
| | | | recv)
* Add client_id and correlation_id to BrokerConnection constructor kwargsDana Powers2015-12-091-3/+7
|
* Add size and correlation id decoding to try/except block in BrokerConnectionDana Powers2015-12-041-4/+5
|
* BrokerConnection:Dana Powers2015-12-041-11/+65
| | | | | | separate send / recv in BrokerConnection improve connection and error handling use different read and write fds for py3
* Add simple BrokerConnection class; add request.RESPONSE_TYPE class varsDana Powers2015-12-041-0/+36
|
* Merge pull request #473 from ecanzonieri/use_unblocking_io_for_aware_requestsDana Powers2015-12-021-0/+5
|\ | | | | Use unblocking io for broker aware requests
| * Unblocking broker aware requestEnrico Canzonieri2015-11-101-0/+5
| |
* | Support retry semantics in MultiProcessConsumerRoss Duggan2015-09-141-0/+4
|/
* Add a few extra docstring comments about thread-safe clients/connectionsDana Powers2015-06-081-3/+5
|
* Use separate module loggers instead of a single 'kafka' loggerDana Powers2015-06-061-1/+2
|
* Returned original tests, rm dirty flag, name fixesViktor Shlapakov2015-02-251-1/+0
|
* Make KafkaConnection copies usable across threadsAlexey Borzenkov2015-02-251-0/+4
|
* Add Sphinx API docsWill Daly2015-01-151-8/+15
|
* Merge pull request #227 from wizzat-feature/py3Dana Powers2014-09-071-4/+10
|\ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Python 3 Support Conflicts: kafka/producer.py test/test_client.py test/test_client_integration.py test/test_codec.py test/test_consumer.py test/test_consumer_integration.py test/test_failover_integration.py test/test_producer.py test/test_producer_integration.py test/test_protocol.py test/test_util.py
| * Fix more tests, only multiprocessing consumer ones remainingBruno Renié2014-09-031-0/+4
| |
| * Make all unit tests pass on py3.3/3.4Bruno Renié2014-09-031-4/+6
| |
* | Minor import cleanupsDana Powers2014-09-041-1/+1
|/
* Add pylint to tox.ini; test both kafka and test; default to error-checking ↵Dana Powers2014-08-251-3/+3
| | | | only; fixup errors; skip kafka/queue.py
* Improve docstrings in kafka.connDana Powers2014-08-191-4/+14
|
* Shutdown socket before closing in kafka.connDana Powers2014-08-191-0/+9
|
* Remove duplicate error handling blocks by moving empty response check into ↵Dana Powers2014-08-191-4/+8
| | | | try block in conn.recv
* Remove self._dirty and check self._sock instead in kafka.connDana Powers2014-08-191-6/+11
|
* socket.sendall should always raise an exception on error; remove extra ↵Dana Powers2014-08-191-6/+7
| | | | return val check in KafkaConnection.send()
* Add some more debug logging to KafkaConnectionDana Powers2014-08-191-0/+7
|
* Initialized _dirty in KafkaConnection __init__() and set _sock to None in ↵Dana Powers2014-08-171-2/+4
| | | | close()
* Call _raise_connection_error in KafkaConnection.reinit() if socket ↵Dana Powers2014-08-171-2/+6
| | | | connection fails
* Support IPv6 hosts and networksAlexey Borzenkov2014-05-091-3/+1
|
* Fix lack of timeout support in KafkaClient and KafkaConnectionmaciejkula2014-04-161-1/+1
|
* Merge branch 'master' into conn_refactorMark Roberts2014-03-181-2/+2
|\
| * Check against basestring instead of str in collect.hosts.Saulius Zemaitaitis2014-03-171-1/+1
| |