summaryrefslogtreecommitdiff
path: root/kafka
Commit message (Collapse)AuthorAgeFilesLines
...
| * | 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
* | | Merge pull request #204 from mdaniel/better-type-errorsDana Powers2014-08-221-1/+8
|\ \ \ | | | | | | | | Better type errors
| * | | Fix write_int and write_short type validationMatthew L Daniel2014-08-221-0/+6
| | | | | | | | | | | | | | | | It will still die, just as before, but it now includes a *helpful* error message
| * | | PEP8 fixesMatthew L Daniel2014-08-221-1/+2
| |/ /
* | | Merge pull request #194 from dpkp/kafka_timeout_errorDana Powers2014-08-222-3/+7
|\ \ \ | | | | | | | | Add KafkaTimeoutError and fix client.ensure_topic_exists
| * | | Add KafkaTimeoutError (used by client.ensure_topic_exists) and add a testDana Powers2014-08-142-3/+7
| |/ /
* | | Add TravisCI config for auto deploymentDavid Arthur2014-08-221-1/+1
|/ / | | | | | | Tags applied to master will now be automatically deployed on PyPI
* | Merge pull request #174 from wizzat/new_topicDana Powers2014-08-102-11/+24
|\ \ | | | | | | Handle New Topic Creation
| * | Handle New Topic CreationMark Roberts2014-05-222-11/+24
| | | | | | | | | | | | | | | Adds ensure_topic_exists to KafkaClient, redirects test case to use that. Fixes #113 and fixes #150.
* | | Set module version in separate VERSION fileDana Powers2014-08-101-1/+3
|/ /
* | Merge pull request #166 from patricklucas/teach_producers_about_compressionDana Powers2014-05-193-22/+59
|\ \ | | | | | | Add 'codec' parameter to Producer
| * | Improve error handling and tests w.r.t. codecsPatrick Lucas2014-05-073-21/+33
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
| * | Merge branch 'teach_producers_about_compression' into producer_compressionMark Roberts2014-05-072-20/+45
| |\ \ | | | | | | | | | | | | | | | | | | | | Conflicts: servers/0.8.0/kafka-src test/test_unit.py
| | * | Add 'codec' parameter to ProducerPatrick Lucas2014-05-032-20/+45
| | | | | | | | | | | | | | | | | | | | Adds a codec parameter to Producer.__init__ that lets the user choose a compression codec to use for all messages sent by it.
* | | | Support IPv6 hosts and networksAlexey Borzenkov2014-05-091-3/+1
|/ / /
* | | Merge branch 'master' into add_testsMark Roberts2014-05-061-1/+15
|\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | kafka/client.py contained duplicate copies of same refactor, merged. Move test/test_integration.py changes into test/test_producer_integration. Conflicts: kafka/client.py servers/0.8.0/kafka-src test/test_integration.py
| * \ \ Merge pull request #139 from alexcb/masterDana Powers2014-05-061-1/+15
| |\ \ \ | | | | | | | | | | SimpleProducer randomization of initial round robin ordering
| | * | | added random_start param to SimpleProducer to enable/disable randomization ↵Alex Couture-Beil2014-04-011-4/+11
| | | | | | | | | | | | | | | | | | | | of the initial partition messages are published to
| | * | | Changed randomization to simply randomize the initial starting partition of ↵Alex Couture-Beil2014-04-011-3/+7
| | | | | | | | | | | | | | | | | | | | the sorted list of partition rather than completely randomizing the initial ordering before round-robin cycling the partitions
| | * | | Modified SimpleProducer to randomize the initial round robin orderingAlex Couture-Beil2014-03-111-1/+4
| | | | | | | | | | | | | | | | | | | | | | | | | of partitions to prevent the first message from always being published to partition 0.
| * | | | Fix lack of timeout support in KafkaClient and KafkaConnectionmaciejkula2014-04-162-2/+2
| | |/ / | |/| |
* | | | Attempt to fix travis build. Decrease complexity of service.py in favor of ↵Mark Roberts2014-05-064-8/+12
| | | | | | | | | | | | | | | | in memory logging. Address code review concerns
* | | | Make commit() check for errors instead of simply assert no errorMark Roberts2014-04-301-1/+1
| | | |
* | | | Make BrokerRequestError a base class, make subclasses for each broker errorMark Roberts2014-04-303-53/+113
| | | |
* | | | Various fixesMark Roberts2014-04-252-2/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Bump version number to 0.9.1 Update readme to show supported Kafka/Python versions Validate arguments in consumer.py, add initial consumer unit test Make service kill() child processes when startup fails Add tests for util.py, fix Python 2.6 specific bug.
* | | | Fix last remaining test by making autocommit more intuitiveMark Roberts2014-04-241-1/+1
| | | |
* | | | Split out kafka version environments, default tox no longer runs any ↵Mark Roberts2014-04-231-11/+11
| | | | | | | | | | | | | | | | integration tests, make skipped integration also skip setupClass, implement rudimentary offset support in consumer.py
* | | | Fix bug in socket timeout per PR #161 by maciejkula, add testMark Roberts2014-04-191-1/+1
| | | |
* | | | Split up and speed up producer based integration testsMark Roberts2014-04-172-1/+2
| | | |
* | | | Refactor away _get_conn_for_broker. Fix bug in _get_connMark Roberts2014-04-091-13/+6
| | | |
* | | | Merge branch 'master' into add_testsMark Roberts2014-04-081-0/+4
|\ \ \ \ | |/ / /
| * | | Commit in seek if autocommitMark Roberts2014-03-271-1/+4
| | | |
| * | | Make seek(); commit(); work without commit discarding the seek changeMark Roberts2014-03-251-0/+1
| | | |
* | | | Reinstate test_integrate, make test_protocol more explicit, create testutilMark Roberts2014-04-081-1/+1
| | | |
* | | | Explicit testing of protocol errors. Make tests more explicit, and start ↵Mark Roberts2014-04-082-4/+7
|/ / / | | | | | | | | | 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-213-26/+35
|\ \ \ | | | | | | | | conn.py performance improvements, make examples work, add another example
| * \ \ Merge branch 'master' into conn_refactorMark Roberts2014-03-182-5/+5
| |\ \ \
| * \ \ \ Merge branch 'master' into conn_refactorMark Roberts2014-02-263-9/+46
| |\ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | Conflicts: example.py
| * | | | | Fix grammar in error stringMark Roberts2014-02-251-1/+1
| | | | | |
| * | | | | Minor refactor in conn.py, update version in __init__.py, add ErrorStringMark Roberts2014-02-253-26/+35
| | | | | |
* | | | | | Merge pull request #109 from mrtheb/developDana Powers2014-03-212-10/+29
|\ \ \ \ \ \ | |_|_|/ / / |/| | | | | TopicAndPartition fix when partition has no leader = -1
| * | | | | Merge branch 'master' into developmrtheb2014-03-175-15/+144
| |\ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | Conflicts: test/test_unit.py
| * | | | | | Changes based on comments by @rdiomar, plus added LeaderUnavailableError for ↵mrtheb2014-02-152-4/+11
| | | | | | | | | | | | | | | | | | | | | | | | | | | | clarity
| * | | | | | check for broker None in send_broker_aware_request (added test for it)mrtheb2014-01-311-5/+14
| | | | | | |
| * | | | | | Merge branch 'master' into developmrtheb2014-01-312-74/+114
| |\ \ \ \ \ \ | | | |_|_|_|/ | | |/| | | |
| * | | | | | Handle cases for partition with leader=-1 (not defined)Marc Labbe2014-01-312-10/+12
| | | | | | |
| * | | | | | added mockmrtheb2014-01-181-3/+4
| | | | | | |
* | | | | | | Check against basestring instead of str in collect.hosts.Saulius Zemaitaitis2014-03-171-1/+1
| |_|/ / / / |/| | | | |