mirror of https://github.com/apache/kafka.git
MINOR: Fix the ZK migration system tests (#14409)
As part of validating 3.6.0 RC0, I ran the ZK migration system tests at the RC tag. Pretty much all of them failed due to recent changes (particularly, disallowing migrations with JBOD). All of the changes here are test fixes, so not a release blocker. ================================================================================ SESSION REPORT (ALL TESTS) ducktape version: 0.11.3 session_id: 2023-09-19--007 run time: 8 minutes 51.147 seconds tests run: 5 passed: 5 flaky: 0 failed: 0 ignored: 0 Reviewers: Luke Chen <showuon@gmail.com>
This commit is contained in:
parent
ae352b6397
commit
01fa95c216
|
@ -183,6 +183,7 @@ public class ActivationRecordsGenerator {
|
|||
.append("'zookeeper.metadata.migration.enable' set to 'false'. ");
|
||||
records.add(ZkMigrationState.POST_MIGRATION.toRecord());
|
||||
} else {
|
||||
// This log message is used in zookeeper_migration_test.py
|
||||
logMessageBuilder
|
||||
.append("Staying in ZK migration mode since 'zookeeper.metadata.migration.enable' ")
|
||||
.append("is still 'true'. ");
|
||||
|
|
|
@ -22,7 +22,7 @@ from ducktape.errors import TimeoutError
|
|||
|
||||
from kafkatest.services.console_consumer import ConsoleConsumer
|
||||
from kafkatest.services.kafka import KafkaService
|
||||
from kafkatest.services.kafka.config_property import CLUSTER_ID
|
||||
from kafkatest.services.kafka.config_property import CLUSTER_ID, LOG_DIRS
|
||||
from kafkatest.services.kafka.quorum import isolated_kraft, ServiceQuorumInfo, zk
|
||||
from kafkatest.services.verifiable_producer import VerifiableProducer
|
||||
from kafkatest.services.zookeeper import ZookeeperService
|
||||
|
@ -57,7 +57,8 @@ class TestMigration(ProduceConsumeValidateTest):
|
|||
allow_zk_with_kraft=True,
|
||||
isolated_kafka=self.kafka,
|
||||
server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()],
|
||||
["zookeeper.metadata.migration.enable", "true"]],
|
||||
["zookeeper.metadata.migration.enable", "true"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]],
|
||||
quorum_info_provider=remote_quorum)
|
||||
controller.start()
|
||||
|
||||
|
@ -95,7 +96,11 @@ class TestMigration(ProduceConsumeValidateTest):
|
|||
version=DEV_BRANCH,
|
||||
quorum_info_provider=zk_quorum,
|
||||
allow_zk_with_kraft=True,
|
||||
server_prop_overrides=[["zookeeper.metadata.migration.enable", "false"]])
|
||||
server_prop_overrides=[
|
||||
["zookeeper.metadata.migration.enable", "false"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]
|
||||
])
|
||||
|
||||
self.kafka.security_protocol = "PLAINTEXT"
|
||||
self.kafka.interbroker_security_protocol = "PLAINTEXT"
|
||||
self.zk.start()
|
||||
|
@ -148,7 +153,10 @@ class TestMigration(ProduceConsumeValidateTest):
|
|||
zk=self.zk,
|
||||
allow_zk_with_kraft=True,
|
||||
version=LATEST_3_4,
|
||||
server_prop_overrides=[["zookeeper.metadata.migration.enable", "false"]],
|
||||
server_prop_overrides=[
|
||||
["zookeeper.metadata.migration.enable", "false"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]
|
||||
],
|
||||
topics={self.topic: {"partitions": self.partitions,
|
||||
"replication-factor": self.replication_factor,
|
||||
'configs': {"min.insync.replicas": 2}}})
|
||||
|
@ -209,14 +217,18 @@ class TestMigration(ProduceConsumeValidateTest):
|
|||
version=LATEST_3_4,
|
||||
quorum_info_provider=zk_quorum,
|
||||
allow_zk_with_kraft=True,
|
||||
server_prop_overrides=[["zookeeper.metadata.migration.enable", "true"]])
|
||||
server_prop_overrides=[
|
||||
["zookeeper.metadata.migration.enable", "true"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]
|
||||
])
|
||||
|
||||
remote_quorum = partial(ServiceQuorumInfo, isolated_kraft)
|
||||
controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=LATEST_3_4,
|
||||
allow_zk_with_kraft=True,
|
||||
isolated_kafka=self.kafka,
|
||||
server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()],
|
||||
["zookeeper.metadata.migration.enable", "true"]],
|
||||
["zookeeper.metadata.migration.enable", "true"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]],
|
||||
quorum_info_provider=remote_quorum)
|
||||
|
||||
self.kafka.security_protocol = "PLAINTEXT"
|
||||
|
@ -259,7 +271,7 @@ class TestMigration(ProduceConsumeValidateTest):
|
|||
try:
|
||||
# Shouldn't have to wait too long to see this log message after startup
|
||||
monitor.wait_until(
|
||||
"Staying in the ZK migration",
|
||||
"Staying in ZK migration",
|
||||
timeout_sec=10.0, backoff_sec=.25,
|
||||
err_msg=""
|
||||
)
|
||||
|
@ -284,14 +296,17 @@ class TestMigration(ProduceConsumeValidateTest):
|
|||
version=DEV_BRANCH,
|
||||
quorum_info_provider=zk_quorum,
|
||||
allow_zk_with_kraft=True,
|
||||
server_prop_overrides=[["zookeeper.metadata.migration.enable", "false"]])
|
||||
server_prop_overrides=[
|
||||
["zookeeper.metadata.migration.enable", "false"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]])
|
||||
|
||||
remote_quorum = partial(ServiceQuorumInfo, isolated_kraft)
|
||||
controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=DEV_BRANCH,
|
||||
allow_zk_with_kraft=True,
|
||||
isolated_kafka=self.kafka,
|
||||
server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()],
|
||||
["zookeeper.metadata.migration.enable", "true"]],
|
||||
["zookeeper.metadata.migration.enable", "true"],
|
||||
[LOG_DIRS, KafkaService.DATA_LOG_DIR_1]],
|
||||
quorum_info_provider=remote_quorum)
|
||||
|
||||
self.kafka.security_protocol = "PLAINTEXT"
|
||||
|
|
Loading…
Reference in New Issue