summaryrefslogtreecommitdiff
path: root/kafka/common.py
Commit message (Collapse)AuthorAgeFilesLines
* Use reflection to avoid multiple errno definitionsAlex Couture-Beil2015-01-101-24/+12
|
* Use configure() to check and set configuration keysDana Powers2014-12-151-0/+4
|
* A simpler kafka consumer:Dana Powers2014-12-151-0/+7
| | | | | | | | | | | | | | | | | | ``` kafka = KafkaConsumer('topic1') for m in kafka: print m kafka = KafkaConsumer('topic1', 'topic2', group_id='my_consumer_group', auto_commit_enable=True, auto_commit_interval_ms=30 * 1000, auto_offset_reset='smallest') for m in kafka: process_message(m) kafka.task_done(m) ```
* Use standard exceptions in client._get_leader_for_partition()Dana Powers2014-09-011-4/+0
| | | | | | - drop custom PartitionUnavailable exception - raise UnknownTopicOrPartitionError or LeaderNotAvailableError - add tests for exception raises
* No more custom LeaderUnavailableError exception -- raise LeaderNotAvailableErrorDana Powers2014-09-011-4/+0
|
* if error code is not recognized in check_error: raise UnknownErrorDana Powers2014-09-011-2/+6
|
* Improve metadata protocol handlingDana Powers2014-09-011-7/+23
| | | | | | | | | | - 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]
* Add KafkaTimeoutError (used by client.ensure_topic_exists) and add a testDana Powers2014-08-141-0/+4
|
* Improve error handling and tests w.r.t. codecsPatrick Lucas2014-05-071-0/+6
| | | | | | | | | | Add function kafka.protocol.create_message_set() that takes a list of payloads and a codec and returns a message set with the desired encoding. Introduce kafka.common.UnsupportedCodecError, raised if an unknown codec is specified. Include a test for the new function.
* Attempt to fix travis build. Decrease complexity of service.py in favor of ↵Mark Roberts2014-05-061-2/+8
| | | | in memory logging. Address code review concerns
* Make BrokerRequestError a base class, make subclasses for each broker errorMark Roberts2014-04-301-25/+89
|
* Explicit testing of protocol errors. Make tests more explicit, and start ↵Mark Roberts2014-04-081-0/+4
| | | | working on intermittent failures in test_encode_fetch_request and test_encode_produc_request
* Merge pull request #134 from wizzat/conn_refactorv0.9.0Dana Powers2014-03-211-15/+21
|\ | | | | conn.py performance improvements, make examples work, add another example
| * Fix grammar in error stringMark Roberts2014-02-251-1/+1
| |
| * Minor refactor in conn.py, update version in __init__.py, add ErrorStringMark Roberts2014-02-251-15/+21
| |
* | Changes based on comments by @rdiomar, plus added LeaderUnavailableError for ↵mrtheb2014-02-151-0/+4
| | | | | | | | clarity
* | Handle cases for partition with leader=-1 (not defined)Marc Labbe2014-01-311-4/+0
|/
* Throw KafkaUnavailableError when no brokers availableThomas Dimson2014-01-131-0/+4
|
* Exception hierarchy, invalidate more md on errorsThomas Dimson2014-01-131-6/+25
|
* make changes to be more fault tolerant: clean up connections, brokers, ↵Jim Lim2013-10-041-0/+5
| | | | | | | | failed_messages - add integration tests for sync producer - add integration tests for async producer w. leadership election - use log.exception
* style: fix whitespacesVetoshkin Nikita2013-10-031-0/+4
|
* Fix #44 Add missing exception classv0.8.0David Arthur2013-09-241-0/+16
| | | | Also move the exceptions to common instead of util
* PEP8-ify most of the filesMahendra M2013-05-291-11/+34
| | | | consumer.py and conn.py will be done later after pending merges
* Big code re-orgDavid Arthur2013-04-021-0/+43