mirror of https://github.com/apache/kafka.git
KAFKA-12756: Update ZooKeeper to v3.6.3 (#10918)
Update the ZooKeeper version to v3.6.3. This requires adding dropwizard as a new dependency. Also, add Kafka v2.8.0 to the ducktape system test image. Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
parent
b1562a90e0
commit
4f5b4c868e
|
|
@ -823,6 +823,9 @@ project(':core') {
|
||||||
implementation libs.scalaLogging
|
implementation libs.scalaLogging
|
||||||
implementation libs.slf4jApi
|
implementation libs.slf4jApi
|
||||||
implementation(libs.zookeeper) {
|
implementation(libs.zookeeper) {
|
||||||
|
// Dropwizard Metrics are required by ZooKeeper as of v3.6.0,
|
||||||
|
// but the library should *not* be used in Kafka code
|
||||||
|
implementation libs.dropwizardMetrics
|
||||||
exclude module: 'slf4j-log4j12'
|
exclude module: 'slf4j-log4j12'
|
||||||
exclude module: 'log4j'
|
exclude module: 'log4j'
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,7 @@
|
||||||
|
|
||||||
<h5><a id="upgrade_300_notable" href="#upgrade_300_notable">Notable changes in 3.0.0</a></h5>
|
<h5><a id="upgrade_300_notable" href="#upgrade_300_notable">Notable changes in 3.0.0</a></h5>
|
||||||
<ul>
|
<ul>
|
||||||
|
<li>ZooKeeper has been upgraded to version 3.6.3.</li>
|
||||||
<li>The release tarball no longer includes test, sources, javadoc and test sources jars. These are still published to the Maven Central repository. </li>
|
<li>The release tarball no longer includes test, sources, javadoc and test sources jars. These are still published to the Maven Central repository. </li>
|
||||||
<li>A number of implementation dependency jars are <a href="https://github.com/apache/kafka/pull/10203">now available in the runtime classpath
|
<li>A number of implementation dependency jars are <a href="https://github.com/apache/kafka/pull/10203">now available in the runtime classpath
|
||||||
instead of compile and runtime classpaths</a>. Compilation errors after the upgrade can be fixed by adding the missing dependency jar(s) explicitly
|
instead of compile and runtime classpaths</a>. Compilation errors after the upgrade can be fixed by adding the missing dependency jar(s) explicitly
|
||||||
|
|
|
||||||
|
|
@ -61,6 +61,7 @@ versions += [
|
||||||
bcpkix: "1.66",
|
bcpkix: "1.66",
|
||||||
checkstyle: "8.36.2",
|
checkstyle: "8.36.2",
|
||||||
commonsCli: "1.4",
|
commonsCli: "1.4",
|
||||||
|
dropwizardMetrics: "3.2.5",
|
||||||
gradle: "7.0.2",
|
gradle: "7.0.2",
|
||||||
grgit: "4.1.0",
|
grgit: "4.1.0",
|
||||||
httpclient: "4.5.13",
|
httpclient: "4.5.13",
|
||||||
|
|
@ -111,7 +112,7 @@ versions += [
|
||||||
snappy: "1.1.8.1",
|
snappy: "1.1.8.1",
|
||||||
spotbugs: "4.2.2",
|
spotbugs: "4.2.2",
|
||||||
zinc: "1.3.5",
|
zinc: "1.3.5",
|
||||||
zookeeper: "3.5.9",
|
zookeeper: "3.6.3",
|
||||||
zstd: "1.5.0-2"
|
zstd: "1.5.0-2"
|
||||||
]
|
]
|
||||||
libs += [
|
libs += [
|
||||||
|
|
@ -171,6 +172,7 @@ libs += [
|
||||||
log4j: "log4j:log4j:$versions.log4j",
|
log4j: "log4j:log4j:$versions.log4j",
|
||||||
lz4: "org.lz4:lz4-java:$versions.lz4",
|
lz4: "org.lz4:lz4-java:$versions.lz4",
|
||||||
metrics: "com.yammer.metrics:metrics-core:$versions.metrics",
|
metrics: "com.yammer.metrics:metrics-core:$versions.metrics",
|
||||||
|
dropwizardMetrics: "io.dropwizard.metrics:metrics-core:$versions.dropwizardMetrics",
|
||||||
mockitoCore: "org.mockito:mockito-core:$versions.mockito",
|
mockitoCore: "org.mockito:mockito-core:$versions.mockito",
|
||||||
mockitoInline: "org.mockito:mockito-inline:$versions.mockito",
|
mockitoInline: "org.mockito:mockito-inline:$versions.mockito",
|
||||||
mockitoJunitJupiter: "org.mockito:mockito-junit-jupiter:$versions.mockito",
|
mockitoJunitJupiter: "org.mockito:mockito-junit-jupiter:$versions.mockito",
|
||||||
|
|
|
||||||
|
|
@ -62,6 +62,7 @@ RUN mkdir -p "/opt/kafka-2.4.1" && chmod a+rw /opt/kafka-2.4.1 && curl -s "$KAFK
|
||||||
RUN mkdir -p "/opt/kafka-2.5.1" && chmod a+rw /opt/kafka-2.5.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.5.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.5.1"
|
RUN mkdir -p "/opt/kafka-2.5.1" && chmod a+rw /opt/kafka-2.5.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.5.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.5.1"
|
||||||
RUN mkdir -p "/opt/kafka-2.6.2" && chmod a+rw /opt/kafka-2.6.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.6.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.6.2"
|
RUN mkdir -p "/opt/kafka-2.6.2" && chmod a+rw /opt/kafka-2.6.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.6.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.6.2"
|
||||||
RUN mkdir -p "/opt/kafka-2.7.1" && chmod a+rw /opt/kafka-2.7.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.7.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.7.1"
|
RUN mkdir -p "/opt/kafka-2.7.1" && chmod a+rw /opt/kafka-2.7.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.7.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.7.1"
|
||||||
|
RUN mkdir -p "/opt/kafka-2.8.0" && chmod a+rw /opt/kafka-2.8.0 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.8.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.8.0"
|
||||||
|
|
||||||
# Streams test dependencies
|
# Streams test dependencies
|
||||||
RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.0.1-test.jar" -o /opt/kafka-0.10.0.1/libs/kafka-streams-0.10.0.1-test.jar
|
RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.0.1-test.jar" -o /opt/kafka-0.10.0.1/libs/kafka-streams-0.10.0.1-test.jar
|
||||||
|
|
@ -78,6 +79,7 @@ RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.4.1-test.jar" -o /opt/kafka-2.4.1/lib
|
||||||
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.5.1-test.jar" -o /opt/kafka-2.5.1/libs/kafka-streams-2.5.1-test.jar
|
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.5.1-test.jar" -o /opt/kafka-2.5.1/libs/kafka-streams-2.5.1-test.jar
|
||||||
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.6.2-test.jar" -o /opt/kafka-2.6.2/libs/kafka-streams-2.6.2-test.jar
|
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.6.2-test.jar" -o /opt/kafka-2.6.2/libs/kafka-streams-2.6.2-test.jar
|
||||||
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.7.1-test.jar" -o /opt/kafka-2.7.1/libs/kafka-streams-2.7.1-test.jar
|
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.7.1-test.jar" -o /opt/kafka-2.7.1/libs/kafka-streams-2.7.1-test.jar
|
||||||
|
RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.8.0-test.jar" -o /opt/kafka-2.8.0/libs/kafka-streams-2.8.0-test.jar
|
||||||
|
|
||||||
# The version of Kibosh to use for testing.
|
# The version of Kibosh to use for testing.
|
||||||
# If you update this, also update vagrant/base.sh
|
# If you update this, also update vagrant/base.sh
|
||||||
|
|
|
||||||
|
|
@ -50,9 +50,13 @@ class KafkaVersionTest(Test):
|
||||||
the other on the current development branch."""
|
the other on the current development branch."""
|
||||||
self.kafka = KafkaService(self.test_context, num_nodes=2, zk=self.zk,
|
self.kafka = KafkaService(self.test_context, num_nodes=2, zk=self.zk,
|
||||||
topics={self.topic: {"partitions": 1, "replication-factor": 2}})
|
topics={self.topic: {"partitions": 1, "replication-factor": 2}})
|
||||||
self.kafka.nodes[1].version = LATEST_0_8_2
|
# Be sure to make node[0] the one with v0.8.2 because the topic will be created using the --zookeeper option
|
||||||
self.kafka.nodes[1].config[config_property.INTER_BROKER_PROTOCOL_VERSION] = "0.8.2.X"
|
# since not all nodes support the --bootstrap-server option; the --zookeeper option is removed as of v3.0,
|
||||||
|
# and the topic will be created against the broker on node[0], so that node has to be the one running the older
|
||||||
|
# version (otherwise the kafka-topics --zookeeper command will fail).
|
||||||
|
self.kafka.nodes[0].version = LATEST_0_8_2
|
||||||
|
self.kafka.nodes[0].config[config_property.INTER_BROKER_PROTOCOL_VERSION] = "0.8.2.X"
|
||||||
self.kafka.start()
|
self.kafka.start()
|
||||||
|
|
||||||
assert is_version(self.kafka.nodes[0], [DEV_BRANCH.vstring], logger=self.logger)
|
assert is_version(self.kafka.nodes[0], [LATEST_0_8_2], logger=self.logger)
|
||||||
assert is_version(self.kafka.nodes[1], [LATEST_0_8_2], logger=self.logger)
|
assert is_version(self.kafka.nodes[1], [DEV_BRANCH.vstring], logger=self.logger)
|
||||||
|
|
|
||||||
|
|
@ -24,7 +24,7 @@ from kafkatest.services.zookeeper import ZookeeperService
|
||||||
from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
|
from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
|
||||||
from kafkatest.utils import is_int
|
from kafkatest.utils import is_int
|
||||||
from kafkatest.utils.remote_account import java_version
|
from kafkatest.utils.remote_account import java_version
|
||||||
from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, 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, V_0_11_0_0, V_2_8_0, DEV_BRANCH, KafkaVersion
|
from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, 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, V_0_11_0_0, V_2_8_0, DEV_BRANCH, KafkaVersion
|
||||||
from kafkatest.services.kafka.util import new_jdk_not_supported
|
from kafkatest.services.kafka.util import new_jdk_not_supported
|
||||||
|
|
||||||
class TestUpgrade(ProduceConsumeValidateTest):
|
class TestUpgrade(ProduceConsumeValidateTest):
|
||||||
|
|
@ -55,7 +55,19 @@ class TestUpgrade(ProduceConsumeValidateTest):
|
||||||
# Not trying to detect a problem here leads to failure in the ensuing Kafka roll, which would be a less
|
# Not trying to detect a problem here leads to failure in the ensuing Kafka roll, which would be a less
|
||||||
# intuitive failure than seeing a problem here, so detect ZooKeeper upgrade problems before involving Kafka.
|
# intuitive failure than seeing a problem here, so detect ZooKeeper upgrade problems before involving Kafka.
|
||||||
self.zk.describe(self.topic)
|
self.zk.describe(self.topic)
|
||||||
self.logger.info("First pass bounce - rolling upgrade")
|
# Do some stuff that exercises the use of ZooKeeper before we upgrade to the latest ZooKeeper client version
|
||||||
|
self.logger.info("First pass bounce - rolling Kafka with old ZooKeeper client")
|
||||||
|
for node in self.kafka.nodes:
|
||||||
|
self.kafka.restart_node(node)
|
||||||
|
topic_cfg = {
|
||||||
|
"topic": "another_topic",
|
||||||
|
"partitions": self.partitions,
|
||||||
|
"replication-factor": self.replication_factor,
|
||||||
|
"configs": {"min.insync.replicas": 2}
|
||||||
|
}
|
||||||
|
self.kafka.create_topic(topic_cfg)
|
||||||
|
|
||||||
|
self.logger.info("Second pass bounce - rolling upgrade")
|
||||||
for node in self.kafka.nodes:
|
for node in self.kafka.nodes:
|
||||||
self.kafka.stop_node(node)
|
self.kafka.stop_node(node)
|
||||||
node.version = DEV_BRANCH
|
node.version = DEV_BRANCH
|
||||||
|
|
@ -64,7 +76,7 @@ class TestUpgrade(ProduceConsumeValidateTest):
|
||||||
self.kafka.start_node(node)
|
self.kafka.start_node(node)
|
||||||
self.wait_until_rejoin()
|
self.wait_until_rejoin()
|
||||||
|
|
||||||
self.logger.info("Second pass bounce - remove inter.broker.protocol.version config")
|
self.logger.info("Third pass bounce - remove inter.broker.protocol.version config")
|
||||||
for node in self.kafka.nodes:
|
for node in self.kafka.nodes:
|
||||||
self.kafka.stop_node(node)
|
self.kafka.stop_node(node)
|
||||||
del node.config[config_property.INTER_BROKER_PROTOCOL_VERSION]
|
del node.config[config_property.INTER_BROKER_PROTOCOL_VERSION]
|
||||||
|
|
@ -76,6 +88,9 @@ class TestUpgrade(ProduceConsumeValidateTest):
|
||||||
self.wait_until_rejoin()
|
self.wait_until_rejoin()
|
||||||
|
|
||||||
@cluster(num_nodes=6)
|
@cluster(num_nodes=6)
|
||||||
|
@parametrize(from_kafka_version=str(LATEST_2_8), to_message_format_version=None, compression_types=["none"])
|
||||||
|
@parametrize(from_kafka_version=str(LATEST_2_8), to_message_format_version=None, compression_types=["lz4"])
|
||||||
|
@parametrize(from_kafka_version=str(LATEST_2_8), to_message_format_version=None, compression_types=["snappy"])
|
||||||
@parametrize(from_kafka_version=str(LATEST_2_7), to_message_format_version=None, compression_types=["none"])
|
@parametrize(from_kafka_version=str(LATEST_2_7), to_message_format_version=None, compression_types=["none"])
|
||||||
@parametrize(from_kafka_version=str(LATEST_2_7), to_message_format_version=None, compression_types=["lz4"])
|
@parametrize(from_kafka_version=str(LATEST_2_7), to_message_format_version=None, compression_types=["lz4"])
|
||||||
@parametrize(from_kafka_version=str(LATEST_2_7), to_message_format_version=None, compression_types=["snappy"])
|
@parametrize(from_kafka_version=str(LATEST_2_7), to_message_format_version=None, compression_types=["snappy"])
|
||||||
|
|
|
||||||
|
|
@ -146,6 +146,8 @@ get_kafka 2.6.2 2.12
|
||||||
chmod a+rw /opt/kafka-2.6.2
|
chmod a+rw /opt/kafka-2.6.2
|
||||||
get_kafka 2.7.1 2.12
|
get_kafka 2.7.1 2.12
|
||||||
chmod a+rw /opt/kafka-2.7.1
|
chmod a+rw /opt/kafka-2.7.1
|
||||||
|
get_kafka 2.8.0 2.12
|
||||||
|
chmod a+rw /opt/kafka-2.8.0
|
||||||
|
|
||||||
# For EC2 nodes, we want to use /mnt, which should have the local disk. On local
|
# For EC2 nodes, we want to use /mnt, which should have the local disk. On local
|
||||||
# VMs, we can just create it if it doesn't exist and use it like we'd use
|
# VMs, we can just create it if it doesn't exist and use it like we'd use
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue