diff options
-rw-r--r-- | .gitignore | 2 | ||||
-rw-r--r-- | README.rst | 9 | ||||
-rw-r--r-- | docs/index.rst | 9 | ||||
-rw-r--r-- | docs/usage.rst | 7 | ||||
-rw-r--r-- | kafka/client.py | 3 | ||||
-rw-r--r-- | kafka/consumer/group.py | 9 | ||||
-rw-r--r-- | kafka/consumer/subscription_state.py | 5 | ||||
-rw-r--r-- | kafka/version.py | 2 | ||||
-rw-r--r-- | servers/0.10.0.0/resources/kafka.properties | 133 | ||||
-rw-r--r-- | servers/0.10.0.0/resources/log4j.properties | 25 | ||||
-rw-r--r-- | servers/0.10.0.0/resources/zookeeper.properties | 21 | ||||
-rw-r--r-- | test/test_client.py | 6 | ||||
-rw-r--r-- | test/test_consumer_group.py | 30 |
13 files changed, 244 insertions, 17 deletions
@@ -5,7 +5,7 @@ build dist MANIFEST env -servers/*/kafka-bin +servers/*/kafka-bin* .coverage* .noseids docs/_build @@ -62,7 +62,6 @@ that expose basic message attributes: topic, partition, offset, key, and value: >>> consumer = KafkaConsumer(value_deserializer=msgpack.dumps) >>> consumer.subscribe(['msgpackfoo']) >>> for msg in consumer: -... msg = next(consumer) ... assert isinstance(msg.value, dict) @@ -76,9 +75,13 @@ for more details. >>> from kafka import KafkaProducer >>> producer = KafkaProducer(bootstrap_servers='localhost:1234') ->>> producer.send('foobar', b'some_message_bytes') +>>> for _ in range(100): +... producer.send('foobar', b'some_message_bytes') ->>> # Blocking send +>>> # Block until all pending messages are sent +>>> producer.flush() + +>>> # Block until a single message is sent (or timeout) >>> producer.send('foobar', b'another_message').get(timeout=60) >>> # Use a key for hashed-partitioning diff --git a/docs/index.rst b/docs/index.rst index d8f826a..b98f119 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -61,7 +61,6 @@ that expose basic message attributes: topic, partition, offset, key, and value: >>> consumer = KafkaConsumer(value_deserializer=msgpack.loads) >>> consumer.subscribe(['msgpackfoo']) >>> for msg in consumer: -... msg = next(consumer) ... assert isinstance(msg.value, dict) @@ -74,9 +73,13 @@ client. See `KafkaProducer <apidoc/KafkaProducer.html>`_ for more details. >>> from kafka import KafkaProducer >>> producer = KafkaProducer(bootstrap_servers='localhost:1234') ->>> producer.send('foobar', b'some_message_bytes') +>>> for _ in range(100): +... producer.send('foobar', b'some_message_bytes') ->>> # Blocking send +>>> # Block until all pending messages are sent +>>> producer.flush() + +>>> # Block until a single message is sent (or timeout) >>> producer.send('foobar', b'another_message').get(timeout=60) >>> # Use a key for hashed-partitioning diff --git a/docs/usage.rst b/docs/usage.rst index d48cc0a..85fc44f 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -87,5 +87,12 @@ KafkaProducer producer = KafkaProducer(value_serializer=lambda m: json.dumps(m).encode('ascii')) producer.send('json-topic', {'key': 'value'}) + # produce asynchronously + for _ in range(100): + producer.send('my-topic', b'msg') + + # block until all async messages are sent + producer.flush() + # configure multiple retries producer = KafkaProducer(retries=5) diff --git a/kafka/client.py b/kafka/client.py index 4b5a043..11f54eb 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -169,7 +169,8 @@ class SimpleClient(object): for payload in payloads: try: leader = self._get_leader_for_partition(payload.topic, payload.partition) - except KafkaUnavailableError: + except (KafkaUnavailableError, LeaderNotAvailableError, + UnknownTopicOrPartitionError): leader = None payloads_by_broker[leader].append(payload) return dict(payloads_by_broker) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 32a4a49..9172040 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -149,6 +149,7 @@ class KafkaConsumer(six.Iterator): 'auto_offset_reset': 'latest', 'enable_auto_commit': True, 'auto_commit_interval_ms': 5000, + 'default_offset_commit_callback': lambda offsets, response: True, 'check_crcs': True, 'metadata_max_age_ms': 5 * 60 * 1000, 'partition_assignment_strategy': (RangePartitionAssignor, RoundRobinPartitionAssignor), @@ -527,6 +528,14 @@ class KafkaConsumer(six.Iterator): log.debug("Pausing partition %s", partition) self._subscription.pause(partition) + def paused(self): + """Get the partitions that were previously paused by a call to pause(). + + Returns: + set: {partition (TopicPartition), ...} + """ + return self._subscription.paused_partitions() + def resume(self, *partitions): """Resume fetching from the specified (paused) partitions. diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index a4043a1..3d170ae 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -265,6 +265,11 @@ class SubscriptionState(object): """Return set of TopicPartitions in current assignment.""" return set(self.assignment.keys()) + def paused_partitions(self): + """Return current set of paused TopicPartitions.""" + return set(partition for partition in self.assignment + if self.is_paused(partition)) + def fetchable_partitions(self): """Return set of TopicPartitions that should be Fetched.""" fetchable = set() diff --git a/kafka/version.py b/kafka/version.py index a6221b3..6c69c82 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '1.0.2' +__version__ = '1.0.3-dev' diff --git a/servers/0.10.0.0/resources/kafka.properties b/servers/0.10.0.0/resources/kafka.properties new file mode 100644 index 0000000..2fd9c54 --- /dev/null +++ b/servers/0.10.0.0/resources/kafka.properties @@ -0,0 +1,133 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id={broker_id} + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port={port} + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +host.name={host} + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name=<hostname routable by clients> + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port=<port accessible by clients> + +# The number of threads handling network requests +num.network.threads=3 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=102400 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs={tmp_dir}/data + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions={partitions} +default.replication.factor={replicas} + +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +# tune down offset topics to reduce setup time in tests +offsets.commit.timeout.ms=500 +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=2 + +# Allow shorter session timeouts for tests +group.min.session.timeout.ms=1000 + + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=30000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/0.10.0.0/resources/log4j.properties b/servers/0.10.0.0/resources/log4j.properties new file mode 100644 index 0000000..b0b76aa --- /dev/null +++ b/servers/0.10.0.0/resources/log4j.properties @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=INFO, stdout, logfile + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.appender.logfile=org.apache.log4j.FileAppender +log4j.appender.logfile.File=${kafka.logs.dir}/server.log +log4j.appender.logfile.layout=org.apache.log4j.PatternLayout +log4j.appender.logfile.layout.ConversionPattern=[%d] %p %m (%c)%n diff --git a/servers/0.10.0.0/resources/zookeeper.properties b/servers/0.10.0.0/resources/zookeeper.properties new file mode 100644 index 0000000..e3fd097 --- /dev/null +++ b/servers/0.10.0.0/resources/zookeeper.properties @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir={tmp_dir} +# the port at which the clients will connect +clientPort={port} +clientPortAddress={host} +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 diff --git a/test/test_client.py b/test/test_client.py index 5a35c83..a53fce1 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -11,7 +11,7 @@ from kafka.common import ( BrokerMetadata, TopicPartition, KafkaUnavailableError, LeaderNotAvailableError, UnknownTopicOrPartitionError, - KafkaTimeoutError, ConnectionError + KafkaTimeoutError, ConnectionError, FailedPayloadsError ) from kafka.conn import KafkaConnection from kafka.future import Future @@ -361,7 +361,7 @@ class TestSimpleClient(unittest.TestCase): "topic_noleader", 0, [create_message("a"), create_message("b")])] - with self.assertRaises(LeaderNotAvailableError): + with self.assertRaises(FailedPayloadsError): client.send_produce_request(requests) @patch('kafka.SimpleClient._get_conn') @@ -386,7 +386,7 @@ class TestSimpleClient(unittest.TestCase): "topic_doesnt_exist", 0, [create_message("a"), create_message("b")])] - with self.assertRaises(UnknownTopicOrPartitionError): + with self.assertRaises(FailedPayloadsError): client.send_produce_request(requests) def test_timeout(self): diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index 34b1be4..5fcfbe2 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -17,10 +17,13 @@ from test.conftest import version from test.testutil import random_string +def get_connect_str(kafka_broker): + return 'localhost:' + str(kafka_broker.port) + + @pytest.fixture def simple_client(kafka_broker): - connect_str = 'localhost:' + str(kafka_broker.port) - return SimpleClient(connect_str) + return SimpleClient(get_connect_str(kafka_broker)) @pytest.fixture @@ -37,8 +40,7 @@ def test_consumer(kafka_broker, version): if version >= (0, 8, 2) and version < (0, 9): topic(simple_client(kafka_broker)) - connect_str = 'localhost:' + str(kafka_broker.port) - consumer = KafkaConsumer(bootstrap_servers=connect_str) + consumer = KafkaConsumer(bootstrap_servers=get_connect_str(kafka_broker)) consumer.poll(500) assert len(consumer._client._conns) > 0 node_id = list(consumer._client._conns.keys())[0] @@ -49,7 +51,7 @@ def test_consumer(kafka_broker, version): @pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") def test_group(kafka_broker, topic): num_partitions = 4 - connect_str = 'localhost:' + str(kafka_broker.port) + connect_str = get_connect_str(kafka_broker) consumers = {} stop = {} threads = {} @@ -120,6 +122,24 @@ def test_group(kafka_broker, topic): threads[c].join() +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_paused(kafka_broker, topic): + consumer = KafkaConsumer(bootstrap_servers=get_connect_str(kafka_broker)) + topics = [TopicPartition(topic, 1)] + consumer.assign(topics) + assert set(topics) == consumer.assignment() + assert set() == consumer.paused() + + consumer.pause(topics[0]) + assert set([topics[0]]) == consumer.paused() + + consumer.resume(topics[0]) + assert set() == consumer.paused() + + consumer.unsubscribe() + assert set() == consumer.paused() + + @pytest.fixture def conn(mocker): conn = mocker.patch('kafka.client_async.BrokerConnection') |