Skip to content

KAFKA-18949: add consumer protocol to testDeleteRecordsAfterCorruptRecords #19317

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 7 commits into from
Apr 3, 2025
Original file line number Diff line number Diff line change
Expand Up @@ -1565,17 +1565,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}

@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
@MethodSource(Array("getTestGroupProtocolParametersClassicGroupProtocolOnly"))
@MethodSource(Array("getTestGroupProtocolParametersAll"))
def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = {
val config = new Properties()
config.put(TopicConfig.SEGMENT_BYTES_CONFIG, "200")
createTopic(topic, numPartitions = 1, replicationFactor = 1, config)

client = createAdminClient

val consumer = createConsumer()
subscribeAndWaitForAssignment(topic, consumer)

val producer = createProducer()
def sendRecords(begin: Int, end: Int) = {
val futures = (begin until end).map( i => {
Expand Down Expand Up @@ -1608,7 +1605,10 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
newContent.flip()
Files.write(logFilePath, newContent.array(), StandardOpenOption.TRUNCATE_EXISTING)

consumer.seekToBeginning(Collections.singletonList(topicPartition))
val overrideConfig = new Properties
overrideConfig.setProperty("auto.offset.reset", "earliest")
val consumer = createConsumer(configOverrides = overrideConfig)
consumer.subscribe(Seq(topic).asJava)
assertEquals("Encountered corrupt message when fetching offset 0 for topic-partition topic-0",
assertThrows(classOf[KafkaException], () => consumer.poll(JDuration.ofMillis(DEFAULT_MAX_WAIT_MS))).getMessage)

Expand Down