summaryrefslogtreecommitdiff
path: root/kafka/client.py
Commit message (Collapse)AuthorAgeFilesLines
...
| * No more custom LeaderUnavailableError exception -- raise LeaderNotAvailableErrorDana Powers2014-09-011-3/+3
| |
| * Rearrange KafkaClient method order to group client metadata methods togetherDana Powers2014-09-011-18/+18
| |
| * Improve metadata protocol handlingDana Powers2014-09-011-9/+21
| | | | | | | | | | | | | | | | | | | | - add MetadataRequest and MetadataResponse namedtuples - add TopicMetadata namedtuple - add error codes to Topic and Partition Metadata - add KafkaClient.send_metadata_request() method - KafkaProtocol.decode_metadata_response changed to return a MetadataResponse object so that it is consistent with server api: [broker_list, topic_list]
* | Fix test errors related to mocksMark Roberts2014-09-031-3/+4
| |
* | Fix more tests, only multiprocessing consumer ones remainingBruno Renié2014-09-031-3/+3
| |
* | client_id in bytesBruno Renié2014-09-031-1/+1
| |
* | Make all unit tests pass on py3.3/3.4Bruno Renié2014-09-031-1/+1
|/
* Add pylint to tox.ini; test both kafka and test; default to error-checking ↵Dana Powers2014-08-251-1/+1
| | | | only; fixup errors; skip kafka/queue.py
* Add KafkaTimeoutError (used by client.ensure_topic_exists) and add a testDana Powers2014-08-141-3/+3
|
* Handle New Topic CreationMark Roberts2014-05-221-9/+19
| | | | | Adds ensure_topic_exists to KafkaClient, redirects test case to use that. Fixes #113 and fixes #150.
* Attempt to fix travis build. Decrease complexity of service.py in favor of ↵Mark Roberts2014-05-061-2/+0
| | | | in memory logging. Address code review concerns
* Make BrokerRequestError a base class, make subclasses for each broker errorMark Roberts2014-04-301-19/+17
|
* Refactor away _get_conn_for_broker. Fix bug in _get_connMark Roberts2014-04-091-13/+6
|
* Merge pull request #134 from wizzat/conn_refactorv0.9.0Dana Powers2014-03-211-3/+3
|\ | | | | conn.py performance improvements, make examples work, add another example
| * Merge branch 'master' into conn_refactorMark Roberts2014-03-181-3/+3
| |\
| * \ Merge branch 'master' into conn_refactorMark Roberts2014-02-261-7/+18
| |\ \ | | | | | | | | | | | | | | | | Conflicts: example.py
| * | | Minor refactor in conn.py, update version in __init__.py, add ErrorStringMark Roberts2014-02-251-3/+3
| | | |
* | | | Merge branch 'master' into developmrtheb2014-03-171-9/+20
|\ \ \ \ | | |_|/ | |/| | | | | | | | | | Conflicts: test/test_unit.py
| * | | If a broker refuses the connection, try the nextstephenarmstrong2014-03-131-3/+3
| | |/ | |/|
| * | Support list (or comma-separated) of hosts (replaces host and port arguments)mrtheb2014-02-091-6/+7
| | |
| * | Merge branch 'master' into multihostsmrtheb2014-01-311-91/+120
| |\ \ | | |/ | | | | | | | | | | | | | | | | | | | | | Conflicts: kafka/client.py kafka/conn.py setup.py test/test_integration.py test/test_unit.py
| * | Allow KafkaClient to take in a list of brokers for bootstrappingMarc Labbe2013-11-141-14/+21
| | |
* | | Changes based on comments by @rdiomar, plus added LeaderUnavailableError for ↵mrtheb2014-02-151-4/+7
| | | | | | | | | | | | clarity
* | | check for broker None in send_broker_aware_request (added test for it)mrtheb2014-01-311-5/+14
| | |
* | | Handle cases for partition with leader=-1 (not defined)Marc Labbe2014-01-311-6/+12
| | |
* | | added mockmrtheb2014-01-181-3/+4
| |/ |/|
* | Merge pull request #107 from rdiomar/fix_default_timeoutsMarc Labbé2014-01-161-2/+6
|\ \ | | | | | | Increase default connection timeout
| * | Change default socket timeout to 120 seconds in both the client and connectionOmar Ghishan2014-01-161-2/+6
| | |
* | | Merge branch 'repr' of https://github.com/mahendra/kafka-python into ↵mrtheb2014-01-141-0/+3
|\ \ \ | |/ / |/| | | | | | | | | | | | | | | | | mahendra-repr Conflicts: kafka/client.py kafka/consumer.py
| * | Add proper string representations for each classMahendra M2013-10-081-1/+4
| |/
* | Throw KafkaUnavailableError when no brokers availableThomas Dimson2014-01-131-2/+2
| |
* | Exception hierarchy, invalidate more md on errorsThomas Dimson2014-01-131-80/+88
| |
* | Change BufferUnderflowError to ConnectionError in conn._read_bytes()Omar Ghishan2014-01-081-3/+2
| | | | | | Both errors are handled the same way when raised and caught, so this makes sense.
* | Fix client error handlingOmar Ghishan2014-01-061-5/+17
| | | | | | | | | | This differentiates between errors that occur when sending the request and receiving the response, and adds BufferUnderflowError handling.
* | * Guarantee reading the expected number of bytes from the socket every timeOmar Ghishan2014-01-061-4/+3
| | | | | | | | | | | | | | | | | | | | | | | | * Remove bufsize from client and conn, since they're not actually enforced Notes: This commit changes behavior a bit by raising a BufferUnderflowError when no data is received for the message size rather than a ConnectionError. Since bufsize in the socket is not actually enforced, but it is used by the consumer when creating requests, moving it there until a better solution is implemented.
* | Allow customizing socket timeouts.Omar Ghishan2014-01-061-3/+4
|/ | | | | | Previously, if you try to consume a message with a timeout greater than 10 seconds, but you don't receive data in those 10 seconds, a socket.timeout exception is raised. This allows a higher socket timeout to be set, or even None for no timeout.
* Merge branch 'master' into prod-windowsMahendra M2013-10-081-6/+18
|\ | | | | | | | | Conflicts: kafka/producer.py
| * make changes to be more fault tolerant: clean up connections, brokers, ↵Jim Lim2013-10-041-6/+18
| | | | | | | | | | | | | | | | failed_messages - add integration tests for sync producer - add integration tests for async producer w. leadership election - use log.exception
* | Ensure that async producer works in windows. Fixes #46Mahendra M2013-10-071-0/+11
|/ | | | | | | | | | | | | | | As per the multiprocessing module's documentation, the objects passed to the Process() class must be pickle-able in Windows. So, the Async producer did not work in windows. To fix this we have to ensure that code which uses multiprocessing has to follow certain rules * The target=func should not be a member function * We cannot pass objects like socket() to multiprocessing This ticket fixes these issues. For KafkaClient and KafkaConnection objects, we make copies of the object and reinit() them inside the child processes.
* Test fixes after flake8 runmrtheb2013-10-031-1/+2
|
* flake8 pass (pep8 and pyflakes)mrtheb2013-10-031-21/+20
|
* style: fix camelCase variable namesVetoshkin Nikita2013-10-031-3/+3
| | | | | | Conflicts: kafka/util.py
* style: use triple quotes for docstringsVetoshkin Nikita2013-10-031-2/+6
|
* style: fix whitespacesVetoshkin Nikita2013-10-031-1/+1
|
* allow a client id to be passed to the clientJim Lim2013-09-241-4/+5
|
* Merge branch 'issue-35'David Arthur2013-07-261-3/+11
|\ | | | | | | | | | | | | Conflicts: kafka/__init__.py kafka/consumer.py test/test_integration.py
| * Re-init the sockets in the new processMahendra M2013-06-271-0/+4
| |
| * Implement blocking get_messages for SimpleConsumerMahendra M2013-06-251-3/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | The implementation is done by using simple options to Kafka Fetch Request Also in the SimpleConsumer iterator, update the offset before the message is yielded. This is so that the consumer state is not lost if certain cases. For eg: the message is yielded and consumed by the caller, but the caller does not come back into the generator again. The message will be consumed but the status is not updated in the consumer
* | Make the default case as 'ack on local write'Mahendra M2013-06-201-2/+11
|/ | | | | | | Also, ensure that the case of 'no-acks' works fine In conn.send(), do not wait for the response. Wait for it only on conn.recv(). This behaviour is fine now since the connection is not shared among consumer threads etc.
* Fix bugs and testingMahendra M2013-06-131-0/+5
| | | | | | * Ensure that round-robin partitioner works fine * _load_metadata_for_topics() would cause duplicate and stale entries in self.topic_partitions. Fix this