KAFKA-19042 Move ProducerSendWhileDeletionTest to client-integration-tests module (#19971)

Use Java to rewrite ProducerSendWhileDeletionTest by new test infra and
move it to client-integration-tests module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Ken Huang 2025-06-18 00:47:30 +08:00 committed by GitHub
parent 183ad524dc
commit 91ce182ec5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 302 additions and 158 deletions

View File

@ -0,0 +1,302 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.producer;
import kafka.server.KafkaBroker;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.NewPartitionReassignment;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.test.ClusterInstance;
import org.apache.kafka.common.test.TestUtils;
import org.apache.kafka.common.test.api.ClusterConfigProperty;
import org.apache.kafka.common.test.api.ClusterTest;
import org.apache.kafka.common.test.api.ClusterTestDefaults;
import org.apache.kafka.common.test.api.Type;
import org.apache.kafka.server.config.ServerLogConfigs;
import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile;
import org.apache.kafka.storage.internals.log.UnifiedLog;
import java.io.File;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerSendWhileDeletionTest.BROKER_COUNT;
import static org.apache.kafka.server.config.ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG;
import static org.apache.kafka.server.config.ServerLogConfigs.NUM_PARTITIONS_CONFIG;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
@ClusterTestDefaults(
types = {Type.KRAFT},
brokers = BROKER_COUNT,
serverProperties = {
@ClusterConfigProperty(key = NUM_PARTITIONS_CONFIG, value = "2"),
@ClusterConfigProperty(key = DEFAULT_REPLICATION_FACTOR_CONFIG, value = "2"),
@ClusterConfigProperty(key = AUTO_LEADER_REBALANCE_ENABLE_CONFIG, value = "false"),
@ClusterConfigProperty(key = "log.segment.delete.delay.ms", value = "1000"),
@ClusterConfigProperty(key = ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, value = "100")
}
)
public class ProducerSendWhileDeletionTest {
public static final int BROKER_COUNT = 2;
private static final int DEFAULT_LINGER_MS = 5;
private final int numRecords = 10;
private final String topic = "topic";
private final ClusterInstance cluster;
public ProducerSendWhileDeletionTest(ClusterInstance cluster) {
this.cluster = cluster;
}
/**
* Tests that Producer gets self-recovered when a topic is deleted mid-way of produce.
* <p>
* Producer will attempt to send messages to the partition specified in each record, and should
* succeed as long as the partition is included in the metadata.
*/
@ClusterTest
public void testSendWithTopicDeletionMidWay() throws Exception {
try (var admin = cluster.admin();
var producer = createProducer()
) {
// Create topic with leader as 0 for the 2 partitions.
var topicAssignment = Map.of(
0, List.of(0, 1),
1, List.of(0, 1)
);
admin.createTopics(List.of(new NewTopic(topic, topicAssignment)));
// Change leader to 1 for both the partitions to increase leader epoch from 0 -> 1
var reassignment = Map.of(
new TopicPartition(topic, 0), Optional.of(new NewPartitionReassignment(List.of(1, 0))),
new TopicPartition(topic, 1), Optional.of(new NewPartitionReassignment(List.of(1, 0)))
);
admin.alterPartitionReassignments(reassignment).all().get();
for (var i = 1; i <= numRecords; i++) {
var resp = producer.send(
new ProducerRecord<>(topic, null, ("value" + i).getBytes())
).get();
assertEquals(topic, resp.topic());
}
// Start topic deletion
admin.deleteTopics(List.of(topic)).all().get();
// Verify that the topic is deleted when no metadata request comes in
verifyTopicDeletion();
// Producer should be able to send messages even after topic gets deleted and auto-created
var finalResp = producer.send(new ProducerRecord<>(topic, null, "value".getBytes())).get();
assertEquals(topic, finalResp.topic());
}
}
/**
* Tests that Producer produce to new topic id after recreation.
* <p>
* Producer will attempt to send messages to the partition specified in each record, and should
* succeed as long as the metadata has been updated with new topic id.
*/
@ClusterTest
public void testSendWithRecreatedTopic() throws Exception {
try (var admin = cluster.admin();
var producer = createProducer()
) {
cluster.createTopic(topic, 1, (short) 1);
var topicId = topicMetadata().topicId();
for (int i = 1; i <= numRecords; i++) {
var resp = producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes())).get();
assertEquals(topic, resp.topic());
}
// Start topic deletion
admin.deleteTopics(List.of(topic)).all().get();
// Verify that the topic is deleted when no metadata request comes in
verifyTopicDeletion();
cluster.createTopic(topic, 1, (short) 1);
assertNotEquals(topicId, topicMetadata().topicId());
// Producer should be able to send messages even after topic gets recreated
var recordMetadata = producer.send(new ProducerRecord<>(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get();
assertEquals(topic, recordMetadata.topic());
assertEquals(0, recordMetadata.offset());
}
}
@ClusterTest
public void testSendWithTopicReassignmentIsMidWay() throws Exception {
var partition0 = new TopicPartition(topic, 0);
try (var admin = cluster.admin();
var producer = createProducer()
) {
// Create topic with leader as 0 for the 1 partition.
admin.createTopics(List.of(new NewTopic(topic, Map.of(0, List.of(0)))));
assertLeader(partition0, 0);
var topicDetails = topicMetadata();
for (var i = 1; i <= numRecords; i++) {
var resp = producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes())).get();
assertEquals(topic, resp.topic());
}
var reassignment = Map.of(
partition0, Optional.of(new NewPartitionReassignment(List.of(1)))
);
// Change replica assignment from 0 to 1. Leadership moves to 1.
admin.alterPartitionReassignments(reassignment).all().get();
assertLeader(partition0, 1);
assertEquals(topicDetails.topicId(), topicMetadata().topicId());
// Producer should be able to send messages even after topic gets reassigned
var recordMetadata = producer.send(new ProducerRecord<>(topic, null, "value".getBytes())).get();
assertEquals(topic, recordMetadata.topic());
}
}
private Producer<String, byte[]> createProducer() {
return cluster.producer(Map.of(
MAX_BLOCK_MS_CONFIG, 5000L,
REQUEST_TIMEOUT_MS_CONFIG, 10000,
DELIVERY_TIMEOUT_MS_CONFIG, 10000 + DEFAULT_LINGER_MS,
KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()
));
}
private void verifyTopicDeletion() throws InterruptedException {
var topicPartitions = List.of(
new TopicPartition(topic, 0),
new TopicPartition(topic, 1)
);
// ensure that the topic-partition has been deleted from all brokers' replica managers
TestUtils.waitForCondition(() ->
cluster.brokers().values().stream()
.allMatch(broker -> topicPartitions.stream()
.allMatch(tp -> broker.replicaManager().onlinePartition(tp).isEmpty())
), "Replica manager's should have deleted all of this topic's partitions");
// ensure that logs from all replicas are deleted
TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
.allMatch(broker -> topicPartitions.stream()
.allMatch(tp -> broker.logManager().getLog(tp, false).isEmpty())
), "Replica logs not deleted after delete topic is complete");
// ensure that topic is removed from all cleaner offsets
TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
.allMatch(broker -> topicPartitions.stream()
.allMatch(tp -> partitionNotInCheckpoint(broker, tp))
), "Cleaner offset for deleted partition should have been removed");
TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
.allMatch(broker -> broker.config().logDirs().stream()
.allMatch(logDir -> topicPartitions.stream().noneMatch(tp ->
new File(logDir, tp.topic() + "-" + tp.partition()).exists())
)
), "Failed to soft-delete the data to a delete directory");
TestUtils.waitForCondition(() -> cluster.brokers().values().stream()
.allMatch(broker -> broker.config().logDirs().stream()
.allMatch(logDir -> deletionDirectoriesAbsent(logDir, topicPartitions))
), "Failed to hard-delete the delete directory");
}
private boolean partitionNotInCheckpoint(KafkaBroker broker, TopicPartition tp) {
List<File> liveLogDirs = new ArrayList<>();
broker.logManager().liveLogDirs().foreach(liveLogDirs::add);
var checkpoints = liveLogDirs.stream().map(logDir -> {
try {
return new OffsetCheckpointFile(new File(logDir, "cleaner-offset-checkpoint"), null).read();
} catch (Exception e) {
return new HashMap<TopicPartition, Long>();
}
}).toList();
return checkpoints.stream().noneMatch(checkpointsPerLogDir ->
checkpointsPerLogDir.containsKey(tp));
}
private boolean deletionDirectoriesAbsent(String logDir, List<TopicPartition> topicPartitions) {
var directoryNames = new File(logDir).list();
if (directoryNames == null) {
return true;
}
return topicPartitions.stream().allMatch(tp ->
Arrays.stream(directoryNames).noneMatch(directoryName ->
directoryName.startsWith(tp.topic() + "-" + tp.partition()) &&
directoryName.endsWith(UnifiedLog.DELETE_DIR_SUFFIX)));
}
private TopicDescription topicMetadata() throws Exception {
try (var admin = cluster.admin()) {
return admin.describeTopics(List.of(topic))
.allTopicNames()
.get()
.get(topic);
}
}
private void assertLeader(TopicPartition topicPartition, Integer expectedLeaderOpt) throws InterruptedException {
try (var admin = cluster.admin()) {
TestUtils.waitForCondition(() -> {
try {
Optional<Integer> currentLeader = getCurrentLeader(admin, topicPartition);
return currentLeader.equals(Optional.of(expectedLeaderOpt));
} catch (Exception e) {
if (e.getCause() instanceof UnknownTopicOrPartitionException) {
return false;
}
throw new RuntimeException(e);
}
}, "Waiting for leader to become " + expectedLeaderOpt);
}
}
private Optional<Integer> getCurrentLeader(Admin admin, TopicPartition topicPartition) throws Exception {
return admin.describeTopics(List.of(topicPartition.topic()))
.allTopicNames()
.get()
.get(topicPartition.topic())
.partitions()
.stream()
.filter(p -> p.partition() == topicPartition.partition())
.findFirst()
.map(TopicPartitionInfo::leader)
.map(Node::id);
}
}

View File

@ -1,158 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.{Admin, NewPartitionReassignment, TopicDescription}
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, RecordMetadata}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals}
import org.junit.jupiter.api.Test
import java.nio.charset.StandardCharsets
import java.util
import java.util.Optional
class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
val producerCount: Int = 1
val brokerCount: Int = 2
val defaultLingerMs: Int = 5
serverConfig.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 2.toString)
serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString)
serverConfig.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString)
producerConfig.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 5000L.toString)
producerConfig.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 10000.toString)
producerConfig.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, (10000 + defaultLingerMs).toString)
/**
* Tests that Producer gets self-recovered when a topic is deleted mid-way of produce.
*
* Producer will attempt to send messages to the partition specified in each record, and should
* succeed as long as the partition is included in the metadata.
*/
@Test
def testSendWithTopicDeletionMidWay(): Unit = {
val numRecords = 10
val topic = "topic"
// Create topic with leader as 0 for the 2 partitions.
createTopicWithAssignment(topic, Map(0 -> Seq(0, 1), 1 -> Seq(0, 1)))
val reassignment = util.Map.of(
new TopicPartition(topic, 0), Optional.of(new NewPartitionReassignment(util.List.of(1, 0))),
new TopicPartition(topic, 1), Optional.of(new NewPartitionReassignment(util.List.of(1, 0)))
)
// Change leader to 1 for both the partitions to increase leader epoch from 0 -> 1
val admin = createAdminClient()
admin.alterPartitionReassignments(reassignment).all().get()
val producer = createProducer()
(1 to numRecords).foreach { i =>
val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get
assertEquals(topic, resp.topic())
}
// Start topic deletion
deleteTopic(topic, listenerName)
// Verify that the topic is deleted when no metadata request comes in
TestUtils.verifyTopicDeletion(topic, 2, brokers)
// Producer should be able to send messages even after topic gets deleted and auto-created
assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic())
}
/**
* Tests that Producer produce to new topic id after recreation.
*
* Producer will attempt to send messages to the partition specified in each record, and should
* succeed as long as the metadata has been updated with new topic id.
*/
@Test
def testSendWithRecreatedTopic(): Unit = {
val numRecords = 10
val topic = "topic"
createTopic(topic)
val admin = createAdminClient()
val topicId = topicMetadata(admin, topic).topicId()
val producer = createProducer()
(1 to numRecords).foreach { i =>
val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get
assertEquals(topic, resp.topic())
}
// Start topic deletion
deleteTopic(topic, listenerName)
// Verify that the topic is deleted when no metadata request comes in
TestUtils.verifyTopicDeletion(topic, 2, brokers)
createTopic(topic)
assertNotEquals(topicId, topicMetadata(admin, topic).topicId())
// Producer should be able to send messages even after topic gets recreated
val recordMetadata: RecordMetadata = producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get
assertEquals(topic, recordMetadata.topic())
assertEquals(0, recordMetadata.offset())
}
/**
* Tests that Producer produce to topic during reassignment where topic metadata change on broker side.
*
* Producer will attempt to send messages to the partition specified in each record, and should
* succeed as long as the metadata cache on the leader includes the partition topic id.
*/
@Test
def testSendWithTopicReassignmentIsMidWay(): Unit = {
val numRecords = 10
val topic = "topic"
val partition0: TopicPartition = new TopicPartition(topic, 0)
val admin: Admin = createAdminClient()
// Create topic with leader as 0 for the 1 partition.
createTopicWithAssignment(topic, Map(0 -> Seq(0)))
TestUtils.assertLeader(admin, partition0, 0)
val topicDetails = topicMetadata(admin, topic)
val producer = createProducer()
(1 to numRecords).foreach { i =>
val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get
assertEquals(topic, resp.topic())
}
val reassignment = util.Map.of(
partition0, Optional.of(new NewPartitionReassignment(util.List.of(1))),
)
// Change replica assignment from 0 to 1. Leadership moves to 1.
admin.alterPartitionReassignments(reassignment).all().get()
TestUtils.assertLeader(admin, partition0, 1)
assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId())
// Producer should be able to send messages even after topic gets reassigned
assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic())
}
private def topicMetadata(admin: Admin, topic: String): TopicDescription = {
admin.describeTopics(util.List.of(topic)).allTopicNames().get().get(topic)
}
}