Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add newer Kafka versions to the pipeline #2385

Draft
wants to merge 14 commits into
base: master
Choose a base branch
from
2 changes: 2 additions & 0 deletions .github/workflows/python-package.yml
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,8 @@ jobs:
- "2.4.0"
- "2.5.0"
- "2.6.0"
- "3.4.0"
- "3.5.0"
steps:
- name: Checkout the source code
uses: actions/checkout@v3
Expand Down
26 changes: 15 additions & 11 deletions build_integration.sh
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
#!/bin/bash

: ${ALL_RELEASES:="0.8.2.2 0.9.0.1 0.10.1.1 0.10.2.2 0.11.0.3 1.0.2 1.1.1 2.0.1 2.1.1 2.2.1 2.3.0 2.4.0 2.5.0"}
: ${ALL_RELEASES:="0.8.2.2 0.9.0.1 0.10.1.1 0.10.2.2 0.11.0.3 1.0.2 1.1.1 2.0.1 2.1.1 2.2.1 2.3.0 2.4.0 2.5.0 3.4.0 3.5.0"}
: ${SCALA_VERSION:=2.11}
: ${DIST_BASE_URL:=https://archive.apache.org/dist/kafka/}
: ${KAFKA_SRC_GIT:=https://github.com/apache/kafka.git}
Expand All @@ -16,6 +16,14 @@ pushd servers
mkdir -p dist
pushd dist
for kafka in $KAFKA_VERSION; do
if [ "$kafka" == "0.8.0" ]; then
SCALA_VERSION="2.8.0"
elif [ "$kafka" \> "2.4.0" ]; then
SCALA_VERSION="2.12"
else
SCALA_VERSION="2.11"
fi
KAFKA_ARTIFACT="kafka_${SCALA_VERSION}-${kafka}.tgz"
if [ "$kafka" == "trunk" ]; then
if [ ! -d "$kafka" ]; then
git clone $KAFKA_SRC_GIT $kafka
Expand All @@ -33,14 +41,6 @@ pushd servers
echo "-------------------------------------"
echo "Checking kafka binaries for ${kafka}"
echo
if [ "$kafka" == "0.8.0" ]; then
KAFKA_ARTIFACT="kafka_2.8.0-${kafka}.tar.gz"
else if [ "$kafka" \> "2.4.0" ]; then
KAFKA_ARTIFACT="kafka_2.12-${kafka}.tgz"
else
KAFKA_ARTIFACT="kafka_${SCALA_VERSION}-${kafka}.tgz"
fi
fi
if [ ! -f "../$kafka/kafka-bin/bin/kafka-run-class.sh" ]; then
if [ -f "${KAFKA_ARTIFACT}" ]; then
echo "Using cached artifact: ${KAFKA_ARTIFACT}"
Expand All @@ -55,7 +55,8 @@ pushd servers
fi
fi
echo
echo "Extracting kafka ${kafka} binaries"
echo "Extracting kafka ${kafka} binaries from ${KAFKA_ARTIFACT} into $(pwd)/${kafka}"
mkdir -p ../$kafka/
tar xzvf ${KAFKA_ARTIFACT} -C ../$kafka/
rm -rf ../$kafka/kafka-bin
mv ../$kafka/${KAFKA_ARTIFACT/%.t*/} ../$kafka/kafka-bin
Expand All @@ -67,7 +68,10 @@ pushd servers
echo "$kafka is already installed in servers/$kafka/ -- skipping"
fi
fi
echo
if [ ! -d "../$kafka/resources" ]; then
echo "resources not defined -- copying from .default/resources"
cp -r ../.default/resources ../$kafka/
fi
done
popd
popd
147 changes: 147 additions & 0 deletions servers/.default/resources/kafka.properties
Original file line number Diff line number Diff line change
@@ -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.authorizer.AclAuthorizer
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
4 changes: 4 additions & 0 deletions servers/.default/resources/kafka_server_jaas.conf
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
KafkaServer {{
{jaas_config}
}};
Client {{}};
25 changes: 25 additions & 0 deletions servers/.default/resources/log4j.properties
Original file line number Diff line number Diff line change
@@ -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
21 changes: 21 additions & 0 deletions servers/.default/resources/zookeeper.properties
Original file line number Diff line number Diff line change
@@ -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
17 changes: 7 additions & 10 deletions test/fixtures.py
Original file line number Diff line number Diff line change
Expand Up @@ -346,10 +346,9 @@ def _add_scram_user(self):
self.out("Adding SCRAM credentials for user {} to zookeeper.".format(self.broker_user))
args = self.kafka_run_class_args(
"kafka.admin.ConfigCommand",
"--zookeeper",
"%s:%d/%s" % (self.zookeeper.host,
self.zookeeper.port,
self.zk_chroot),
"--bootstrap-server",
"%s:%d" % (self.zookeeper.host,
self.zookeeper.port),
"--alter",
"--entity-type", "users",
"--entity-name", self.broker_user,
Expand Down Expand Up @@ -586,9 +585,8 @@ def _create_topic_via_admin_api(self, topic_name, num_partitions, replication_fa

def _create_topic_via_cli(self, topic_name, num_partitions, replication_factor):
args = self.kafka_run_class_args('kafka.admin.TopicCommand',
'--zookeeper', '%s:%s/%s' % (self.zookeeper.host,
self.zookeeper.port,
self.zk_chroot),
'--bootstrap-server', '%s:%d' % (self.zookeeper.host,
self.zookeeper.port),
'--create',
'--topic', topic_name,
'--partitions', self.partitions \
Expand All @@ -610,9 +608,8 @@ def _create_topic_via_cli(self, topic_name, num_partitions, replication_factor):

def get_topic_names(self):
args = self.kafka_run_class_args('kafka.admin.TopicCommand',
'--zookeeper', '%s:%s/%s' % (self.zookeeper.host,
self.zookeeper.port,
self.zk_chroot),
'--bootstrap-server', '%s:%d' % (self.zookeeper.host,
self.zookeeper.port),
'--list'
)
env = self.kafka_run_class_env()
Expand Down