mirror of https://github.com/apache/kafka.git
KAFKA-18045 Add 0.11, 1.0, 1.1, and 2.0 back to streams_upgrade_test.py (#17876)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
654ebe10f4
commit
d1952e8542
|
|
@ -78,6 +78,12 @@ RUN echo 'PermitUserEnvironment yes' >> /etc/ssh/sshd_config
|
|||
# Install binary test dependencies.
|
||||
# we use the same versions as in vagrant/base.sh
|
||||
ARG KAFKA_MIRROR="https://s3-us-west-2.amazonaws.com/kafka-packages"
|
||||
# The versions between 0.11.0.3 and 2.0.1 are used to run client code, because zookeeper in these versions is not compatible with JDK 17.
|
||||
# See KAFKA-17888 for more details.
|
||||
RUN mkdir -p "/opt/kafka-0.11.0.3" && chmod a+rw /opt/kafka-0.11.0.3 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.3.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.3"
|
||||
RUN mkdir -p "/opt/kafka-1.0.2" && chmod a+rw /opt/kafka-1.0.2 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.0.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.0.2"
|
||||
RUN mkdir -p "/opt/kafka-1.1.1" && chmod a+rw /opt/kafka-1.1.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.1.1"
|
||||
RUN mkdir -p "/opt/kafka-2.0.1" && chmod a+rw /opt/kafka-2.0.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.0.1"
|
||||
RUN mkdir -p "/opt/kafka-2.1.1" && chmod a+rw /opt/kafka-2.1.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.1.1"
|
||||
RUN mkdir -p "/opt/kafka-2.2.2" && chmod a+rw /opt/kafka-2.2.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.2.2"
|
||||
RUN mkdir -p "/opt/kafka-2.3.1" && chmod a+rw /opt/kafka-2.3.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.3.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.3.1"
|
||||
|
|
|
|||
|
|
@ -22,9 +22,10 @@ from kafkatest.services.kafka import KafkaService, quorum
|
|||
from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService, \
|
||||
StreamsUpgradeTestJobRunnerService
|
||||
from kafkatest.tests.streams.utils import extract_generation_from_logs, extract_generation_id
|
||||
from kafkatest.version import LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \
|
||||
from kafkatest.version import (LATEST_0_11, LATEST_1_0, LATEST_1_1,
|
||||
LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \
|
||||
LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, \
|
||||
DEV_BRANCH, DEV_VERSION, KafkaVersion
|
||||
DEV_BRANCH, DEV_VERSION, KafkaVersion)
|
||||
|
||||
# broker 0.10.0 is not compatible with newer Kafka Streams versions
|
||||
# broker 0.10.1 and 0.10.2 do not support headers, as required by suppress() (since v2.2.1)
|
||||
|
|
@ -32,7 +33,8 @@ broker_upgrade_versions = [str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), st
|
|||
str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6),
|
||||
str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(DEV_BRANCH)]
|
||||
|
||||
metadata_2_versions = [str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8),
|
||||
metadata_2_versions = [str(LATEST_0_11), str(LATEST_1_0), str(LATEST_1_1), str(LATEST_2_0),
|
||||
str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8),
|
||||
str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3)]
|
||||
# upgrading from version (2.4...3.3) is broken and only fixed later in 3.3.3 (unreleased) and 3.4.0
|
||||
# -> https://issues.apache.org/jira/browse/KAFKA-14646
|
||||
|
|
@ -120,7 +122,12 @@ class StreamsUpgradeTest(Test):
|
|||
else:
|
||||
extra_properties = {}
|
||||
|
||||
self.set_up_services()
|
||||
broker_version = DEV_BRANCH
|
||||
# the protocol which is used by versions <= LATEST_2_0 are dropped in 4.0, so setting broker version to 3.9
|
||||
if KafkaVersion(from_version) < LATEST_2_1:
|
||||
broker_version = LATEST_3_9
|
||||
|
||||
self.set_up_services(version=broker_version)
|
||||
|
||||
self.driver.start()
|
||||
|
||||
|
|
@ -212,8 +219,8 @@ class StreamsUpgradeTest(Test):
|
|||
|
||||
self.stop_and_await()
|
||||
|
||||
def set_up_services(self):
|
||||
self.kafka = KafkaService(self.test_context, num_nodes=1, zk=None, topics=self.topics)
|
||||
def set_up_services(self, version=DEV_BRANCH):
|
||||
self.kafka = KafkaService(self.test_context, num_nodes=1, zk=None, topics=self.topics, version=version)
|
||||
self.kafka.start()
|
||||
|
||||
self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka)
|
||||
|
|
|
|||
|
|
@ -112,6 +112,22 @@ DEV_VERSION = KafkaVersion("4.0.0-SNAPSHOT")
|
|||
# This should match the LATEST_PRODUCTION version defined in MetadataVersion.java
|
||||
LATEST_STABLE_METADATA_VERSION = "4.0-IV0"
|
||||
|
||||
# 0.11.0.x versions
|
||||
V_0_11_0_3 = KafkaVersion("0.11.0.3")
|
||||
LATEST_0_11 = V_0_11_0_3
|
||||
|
||||
# 1.0.x versions
|
||||
V_1_0_2 = KafkaVersion("1.0.2")
|
||||
LATEST_1_0 = V_1_0_2
|
||||
|
||||
# 1.1.x versions
|
||||
V_1_1_1 = KafkaVersion("1.1.1")
|
||||
LATEST_1_1 = V_1_1_1
|
||||
|
||||
# 2.0.x versions
|
||||
V_2_0_1 = KafkaVersion("2.0.1")
|
||||
LATEST_2_0 = V_2_0_1
|
||||
|
||||
# 2.1.x versions
|
||||
V_2_1_0 = KafkaVersion("2.1.0")
|
||||
V_2_1_1 = KafkaVersion("2.1.1")
|
||||
|
|
|
|||
|
|
@ -114,6 +114,16 @@ apt-get install -y iperf traceroute
|
|||
# We want to use the latest Scala version per Kafka version
|
||||
# Previously we could not pull in Scala 2.12 builds, because Scala 2.12 requires Java 8 and we were running the system
|
||||
# tests with Java 7. We have since switched to Java 8, so 2.0.0 and later use Scala 2.12.
|
||||
# The versions between 0.11.0.3 and 2.0.1 are used to run client code, because zookeeper in these versions is not compatible with JDK 17.
|
||||
# See KAFKA-17888 for more details.
|
||||
get_kafka 0.11.0.3 2.11
|
||||
chmod a+rw /opt/kafka-0.11.0.3
|
||||
get_kafka 1.0.2 2.11
|
||||
chmod a+rw /opt/kafka-1.0.2
|
||||
get_kafka 1.1.1 2.11
|
||||
chmod a+rw /opt/kafka-1.1.1
|
||||
get_kafka 2.0.1 2.12
|
||||
chmod a+rw /opt/kafka-2.0.1
|
||||
get_kafka 2.1.1 2.12
|
||||
chmod a+rw /opt/kafka-2.1.1
|
||||
get_kafka 2.2.2 2.12
|
||||
|
|
|
|||
Loading…
Reference in New Issue