diff options
author | Lourens Naudé <lourens@methodmissing.com> | 2020-11-15 19:50:46 +0000 |
---|---|---|
committer | GitHub <noreply@github.com> | 2020-11-15 11:50:46 -0800 |
commit | 8b8a48ba09677e38088c61ade2dab2e42198ab3f (patch) | |
tree | a213ec3a9fd14df47965daa690f05246be1b451d | |
parent | 6c87155bbd855f6bba1ba30b2b6227e66ea79baa (diff) | |
download | kafka-python-8b8a48ba09677e38088c61ade2dab2e42198ab3f.tar.gz |
Add Kafka 2.6.0 to tests and protocol compatibility matrix (#2162)
* Co-authored-by: Andrew Brown <andrew.brown@shopify.com>
* Co-authored-by: Aaron Brady <aaron.brady@shopify.com>
-rw-r--r-- | .travis.yml | 1 | ||||
-rw-r--r-- | README.rst | 4 | ||||
-rw-r--r-- | docs/compatibility.rst | 4 | ||||
-rw-r--r-- | docs/index.rst | 4 | ||||
-rw-r--r-- | kafka/conn.py | 3 | ||||
-rw-r--r-- | kafka/protocol/__init__.py | 1 | ||||
-rw-r--r-- | kafka/protocol/admin.py | 42 | ||||
-rw-r--r-- | kafka/protocol/types.py | 13 | ||||
-rw-r--r-- | servers/2.6.0/resources/kafka.properties | 147 | ||||
-rw-r--r-- | servers/2.6.0/resources/kafka_server_jaas.conf | 4 | ||||
-rw-r--r-- | servers/2.6.0/resources/log4j.properties | 25 | ||||
-rw-r--r-- | servers/2.6.0/resources/zookeeper.properties | 21 |
12 files changed, 261 insertions, 8 deletions
diff --git a/.travis.yml b/.travis.yml index e837924..21e51f5 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,6 +17,7 @@ env: - KAFKA_VERSION=1.1.1 - KAFKA_VERSION=2.4.0 - KAFKA_VERSION=2.5.0 + - KAFKA_VERSION=2.6.0 addons: apt: @@ -1,7 +1,7 @@ Kafka Python client ------------------------ -.. image:: https://img.shields.io/badge/kafka-2.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-2.6%2C%202.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/en/master/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python @@ -158,4 +158,4 @@ for interacting with kafka brokers via the python repl. This is useful for testing, probing, and general experimentation. The protocol support is leveraged to enable a KafkaClient.check_version() method that probes a kafka broker and attempts to identify which version it is running -(0.8.0 to 2.4+). +(0.8.0 to 2.6+). diff --git a/docs/compatibility.rst b/docs/compatibility.rst index 60b60bc..b3ad006 100644 --- a/docs/compatibility.rst +++ b/docs/compatibility.rst @@ -1,12 +1,12 @@ Compatibility ------------- -.. image:: https://img.shields.io/badge/kafka-2.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-2.6%2C%202.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python -kafka-python is compatible with (and tested against) broker versions 2.5 +kafka-python is compatible with (and tested against) broker versions 2.6 through 0.8.0 . kafka-python is not compatible with the 0.8.2-beta release. Because the kafka server protocol is backwards compatible, kafka-python is diff --git a/docs/index.rst b/docs/index.rst index 18aeb29..536a058 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,7 +1,7 @@ kafka-python ############ -.. image:: https://img.shields.io/badge/kafka-2.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-2.6%2C%202.5%2C%202.4%2C%202.3%2C%202.2%2C%202.1%2C%202.0%2C%201.1%2C%201.0%2C%200.11%2C%200.10%2C%200.9%2C%200.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python @@ -137,7 +137,7 @@ for interacting with kafka brokers via the python repl. This is useful for testing, probing, and general experimentation. The protocol support is leveraged to enable a :meth:`~kafka.KafkaClient.check_version()` method that probes a kafka broker and -attempts to identify which version it is running (0.8.0 to 2.4+). +attempts to identify which version it is running (0.8.0 to 2.6+). .. toctree:: diff --git a/kafka/conn.py b/kafka/conn.py index 5c72875..cac3548 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -24,7 +24,7 @@ import kafka.errors as Errors from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate from kafka.oauth.abstract import AbstractTokenProvider -from kafka.protocol.admin import SaslHandShakeRequest, DescribeAclsRequest_v2 +from kafka.protocol.admin import SaslHandShakeRequest, DescribeAclsRequest_v2, DescribeClientQuotasRequest from kafka.protocol.commit import OffsetFetchRequest from kafka.protocol.offset import OffsetRequest from kafka.protocol.produce import ProduceRequest @@ -1169,6 +1169,7 @@ class BrokerConnection(object): # in reverse order. As soon as we find one that works, return it test_cases = [ # format (<broker version>, <needed struct>) + ((2, 6, 0), DescribeClientQuotasRequest[0]), ((2, 5, 0), DescribeAclsRequest_v2), ((2, 4, 0), ProduceRequest[8]), ((2, 3, 0), FetchRequest[11]), diff --git a/kafka/protocol/__init__.py b/kafka/protocol/__init__.py index 26dcc78..e739b5c 100644 --- a/kafka/protocol/__init__.py +++ b/kafka/protocol/__init__.py @@ -43,4 +43,5 @@ API_KEYS = { 40: 'ExpireDelegationToken', 41: 'DescribeDelegationToken', 42: 'DeleteGroups', + 48: 'DescribeClientQuotas', } diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index f3b691a..63a3327 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -1,7 +1,7 @@ from __future__ import absolute_import from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String +from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String, Float64 class ApiVersionResponse_v0(Response): @@ -923,3 +923,43 @@ DeleteGroupsRequest = [ DeleteGroupsResponse = [ DeleteGroupsResponse_v0, DeleteGroupsResponse_v1 ] + + +class DescribeClientQuotasResponse_v0(Request): + API_KEY = 48 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ('error_message', String('utf-8')), + ('entries', Array( + ('entity', Array( + ('entity_type', String('utf-8')), + ('entity_name', String('utf-8')))), + ('values', Array( + ('name', String('utf-8')), + ('value', Float64))))), + ) + + +class DescribeClientQuotasRequest_v0(Request): + API_KEY = 48 + API_VERSION = 0 + RESPONSE_TYPE = DescribeClientQuotasResponse_v0 + SCHEMA = Schema( + ('components', Array( + ('entity_type', String('utf-8')), + ('match_type', Int8), + ('match', String('utf-8')), + )), + ('strict', Boolean) + ) + + +DescribeClientQuotasRequest = [ + DescribeClientQuotasRequest_v0, +] + +DescribeClientQuotasResponse = [ + DescribeClientQuotasResponse_v0, +] diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index d508b26..ade1bc6 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -77,6 +77,19 @@ class Int64(AbstractType): return _unpack(cls._unpack, data.read(8)) +class Float64(AbstractType): + _pack = struct.Struct('>d').pack + _unpack = struct.Struct('>d').unpack + + @classmethod + def encode(cls, value): + return _pack(cls._pack, value) + + @classmethod + def decode(cls, data): + return _unpack(cls._unpack, data.read(8)) + + class String(AbstractType): def __init__(self, encoding='utf-8'): self.encoding = encoding diff --git a/servers/2.6.0/resources/kafka.properties b/servers/2.6.0/resources/kafka.properties new file mode 100644 index 0000000..5775cfd --- /dev/null +++ b/servers/2.6.0/resources/kafka.properties @@ -0,0 +1,147 @@ +# 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 ############################# + +listeners={transport}://{host}:{port} +security.inter.broker.protocol={transport} + +{sasl_config} + +ssl.keystore.location={ssl_dir}/kafka.server.keystore.jks +ssl.keystore.password=foobar +ssl.key.password=foobar +ssl.truststore.location={ssl_dir}/kafka.server.truststore.jks +ssl.truststore.password=foobar + +authorizer.class.name=kafka.security.auth.SimpleAclAuthorizer +allow.everyone.if.no.acl.found=true + +# The port the socket server listens on +#port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# 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=1 + +# 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/2.6.0/resources/kafka_server_jaas.conf b/servers/2.6.0/resources/kafka_server_jaas.conf new file mode 100644 index 0000000..af4306c --- /dev/null +++ b/servers/2.6.0/resources/kafka_server_jaas.conf @@ -0,0 +1,4 @@ +KafkaServer {{ + {jaas_config} +}}; +Client {{}}; diff --git a/servers/2.6.0/resources/log4j.properties b/servers/2.6.0/resources/log4j.properties new file mode 100644 index 0000000..b0b76aa --- /dev/null +++ b/servers/2.6.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/2.6.0/resources/zookeeper.properties b/servers/2.6.0/resources/zookeeper.properties new file mode 100644 index 0000000..e3fd097 --- /dev/null +++ b/servers/2.6.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 |