|
17 | 17 | from ducktape.mark.resource import cluster
|
18 | 18 | from ducktape.utils.util import wait_until
|
19 | 19 | from kafkatest.services.console_consumer import ConsoleConsumer
|
20 |
| -from kafkatest.services.kafka import KafkaService |
| 20 | +from kafkatest.services.kafka import config_property, KafkaService |
21 | 21 | from kafkatest.services.kafka.quorum import isolated_kraft, combined_kraft
|
22 | 22 | from kafkatest.services.verifiable_producer import VerifiableProducer
|
23 | 23 | from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
|
@@ -138,14 +138,23 @@ def run_upgrade_downgrade(self, starting_kafka_version):
|
138 | 138 | - Perform rolling downgrade.
|
139 | 139 | - Finally, validate that every message acked by the producer was consumed by the consumer.
|
140 | 140 | """
|
| 141 | + |
| 142 | + # Due to compatability issue with version 3.3, we need to use a single folder. Using multiple folders |
| 143 | + # will cause broker to throw InconsistentBrokerMetadataException during startup. |
| 144 | + # see https://github.com/apache/kafka/pull/13130 |
| 145 | + server_prop_overrides = None |
| 146 | + if starting_kafka_version == str(LATEST_3_3): |
| 147 | + server_prop_overrides = [[config_property.LOG_DIRS, "/mnt/kafka/kafka-metadata-logs"], [config_property.METADATA_LOG_DIR, ""]] |
| 148 | + |
141 | 149 | fromKafkaVersion = KafkaVersion(starting_kafka_version)
|
142 | 150 | self.kafka = KafkaService(self.test_context,
|
143 | 151 | num_nodes=3,
|
144 | 152 | zk=None,
|
145 | 153 | version=fromKafkaVersion,
|
146 | 154 | topics={self.topic: {"partitions": self.partitions,
|
147 | 155 | "replication-factor": self.replication_factor,
|
148 |
| - 'configs': {"min.insync.replicas": 2}}}) |
| 156 | + 'configs': {"min.insync.replicas": 2}}}, |
| 157 | + server_prop_overrides = server_prop_overrides) |
149 | 158 | self.kafka.start()
|
150 | 159 | self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka,
|
151 | 160 | self.topic, throughput=self.producer_throughput,
|
|
0 commit comments