Skip to content

Commit e3f385f

Browse files
jolshanpdruley
authored andcommitted
KAFKA-18035: TransactionsTest testBumpTransactionalEpochWithTV2Disabled failed on trunk (apache#18451)
Sometimes we didn't get into abortable state before aborting, so the epoch didn't get bumped. Now we force abortable state with an attempt to send before aborting so the epoch bump occurs as expected. Reviewers: Jeff Kim <[email protected]>
1 parent 4ca29e5 commit e3f385f

File tree

1 file changed

+15
-4
lines changed

1 file changed

+15
-4
lines changed

core/src/test/scala/integration/kafka/api/TransactionsTest.scala

+15-4
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,8 @@ import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue}
2121
import kafka.utils.{TestInfoUtils, TestUtils}
2222
import org.apache.kafka.clients.consumer._
2323
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
24-
import org.apache.kafka.common.TopicPartition
24+
import org.apache.kafka.common.{KafkaException, TopicPartition}
2525
import org.apache.kafka.common.errors.{ConcurrentTransactionsException, InvalidProducerEpochException, ProducerFencedException, TimeoutException}
26-
import org.apache.kafka.common.test.api.Flaky
2726
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
2827
import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig}
2928
import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs}
@@ -693,7 +692,6 @@ class TransactionsTest extends IntegrationTestHarness {
693692
assertThrows(classOf[IllegalStateException], () => producer.initTransactions())
694693
}
695694

696-
@Flaky("KAFKA-18035")
697695
@ParameterizedTest
698696
@CsvSource(Array(
699697
"kraft,classic,false",
@@ -733,6 +731,19 @@ class TransactionsTest extends IntegrationTestHarness {
733731
restartDeadBrokers()
734732

735733
org.apache.kafka.test.TestUtils.assertFutureThrows(failedFuture, classOf[TimeoutException])
734+
// Ensure the producer transitions to abortable_error state.
735+
TestUtils.waitUntilTrue(() => {
736+
var failed = false
737+
try {
738+
producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(testTopic, 0, "3", "3", willBeCommitted = false))
739+
} catch {
740+
case e: Exception =>
741+
if (e.isInstanceOf[KafkaException])
742+
failed = true
743+
}
744+
failed
745+
}, "The send request never failed as expected.")
746+
assertThrows(classOf[KafkaException], () => producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(testTopic, 0, "3", "3", willBeCommitted = false)))
736747
producer.abortTransaction()
737748

738749
producer.beginTransaction()
@@ -755,7 +766,7 @@ class TransactionsTest extends IntegrationTestHarness {
755766
producerStateEntry =
756767
brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)
757768
assertNotNull(producerStateEntry)
758-
assertTrue(producerStateEntry.producerEpoch > initialProducerEpoch)
769+
assertTrue(producerStateEntry.producerEpoch > initialProducerEpoch, "InitialProduceEpoch: " + initialProducerEpoch + " ProducerStateEntry: " + producerStateEntry)
759770
} finally {
760771
producer.close(Duration.ZERO)
761772
}

0 commit comments

Comments
 (0)