Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-11-09 Thread via GitHub


chia7712 merged PR #17579:
URL: https://github.com/apache/kafka/pull/17579


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-11-08 Thread via GitHub


chia7712 commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1835089796


##
test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java:
##
@@ -125,4 +148,131 @@ static void retryOnExceptionWithTimeout(final Runnable 
runnable) throws Interrup
 Thread.sleep(DEFAULT_POLL_INTERVAL_MS);
 }
 }
+
+/**
+ * Wrap a single record log buffer.
+ */
+public static MemoryRecords singletonRecords(byte[] value,
+ byte[] key,
+ Compression codec,
+ long timestamp,
+ byte magicValue) {
+return records(Collections.singletonList(new SimpleRecord(timestamp, 
key, value)), magicValue, codec);
+}
+
+public static MemoryRecords singletonRecords(byte[] value, byte[] key) {
+return singletonRecords(value, key, Compression.NONE, 
RecordBatch.NO_TIMESTAMP, RecordBatch.CURRENT_MAGIC_VALUE);
+}
+
+public static MemoryRecords records(List records,
+byte magicValue,
+Compression codec,
+long producerId,
+short producerEpoch,
+int sequence,
+long baseOffset,
+int partitionLeaderEpoch) {
+int sizeInBytes = DefaultRecordBatch.sizeInBytes(records);
+ByteBuffer buffer = ByteBuffer.allocate(sizeInBytes);
+
+try (MemoryRecordsBuilder builder = MemoryRecords.builder(
+buffer,
+magicValue,
+codec,
+TimestampType.CREATE_TIME,
+baseOffset,
+System.currentTimeMillis(),
+producerId,
+producerEpoch,
+sequence,
+false,
+partitionLeaderEpoch
+)) {
+records.forEach(builder::append);
+return builder.build();
+}
+}
+
+public static MemoryRecords records(List records, byte 
magicValue, Compression codec) {
+return records(records,
+magicValue,
+codec,
+RecordBatch.NO_PRODUCER_ID,
+RecordBatch.NO_PRODUCER_EPOCH,
+RecordBatch.NO_SEQUENCE,
+0L,
+RecordBatch.NO_PARTITION_LEADER_EPOCH);
+}
+
+public static int waitUntilLeaderIsElectedOrChangedWithAdmin(Admin admin,
+ String topic,
+ int 
partitionNumber,
+ long 
timeoutMs) throws Exception {
+GetPartitionLeader getPartitionLeader = (t, p) -> 
Optional.ofNullable(getLeaderFromAdmin(admin, t, p));
+return doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, 
partitionNumber, timeoutMs);
+}
+
+private static Integer getLeaderFromAdmin(Admin admin, String topic, int 
partition) throws Exception {
+TopicDescription topicDescription = 
admin.describeTopics(Collections.singletonList(topic)).allTopicNames().get().get(topic);
+return topicDescription.partitions().stream()
+.filter(partitionInfo -> partitionInfo.partition() == partition)
+.findFirst()
+.map(partitionInfo -> partitionInfo.leader().id() == 
Node.noNode().id() ? null : partitionInfo.leader().id())
+.orElse(null);
+}
+
+private static int doWaitUntilLeaderIsElectedOrChanged(GetPartitionLeader 
getPartitionLeader,

Review Comment:
   Could you please replace `GetPartitionLeader` by `BiFunction>`?



##
test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java:
##
@@ -225,4 +235,53 @@ default void waitAcls(AclBindingFilter filter, 
Collection en
 }
 }
 
+default void waitForTopicDeletion(String topic, int numPartitions) throws 
Exception {

Review Comment:
   Could you please move all checks to `ClusterInstance#waitForTopic`? those 
checks should be enabled when the input `partitions` is equal to 0



##
test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java:
##
@@ -125,4 +148,131 @@ static void retryOnExceptionWithTimeout(final Runnable 
runnable) throws Interrup
 Thread.sleep(DEFAULT_POLL_INTERVAL_MS);
 }
 }
+
+/**
+ * Wrap a single record log buffer.
+ */
+public static MemoryRecords singletonRecords(byte[] value,
+ byte[] key,
+ Compression codec,
+ long timestamp,
+   

Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-11-07 Thread via GitHub


frankvicky commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1833634453


##
test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java:
##
@@ -125,4 +148,131 @@ static void retryOnExceptionWithTimeout(final Runnable 
runnable) throws Interrup
 Thread.sleep(DEFAULT_POLL_INTERVAL_MS);
 }
 }
+
+/**
+ * Wrap a single record log buffer.
+ */
+public static MemoryRecords singletonRecords(byte[] value,

Review Comment:
   These helper methods come from `TestUtils.scala`. In the Scala version, they 
are widely used across many test classes. Considering that we will eventually 
drop Scala, I think we could retain these helper methods in `TestUtils.java`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-11-07 Thread via GitHub


chia7712 commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1833252842


##
test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java:
##
@@ -125,4 +148,131 @@ static void retryOnExceptionWithTimeout(final Runnable 
runnable) throws Interrup
 Thread.sleep(DEFAULT_POLL_INTERVAL_MS);
 }
 }
+
+/**
+ * Wrap a single record log buffer.
+ */
+public static MemoryRecords singletonRecords(byte[] value,

Review Comment:
   If those helpers are used by `DeleteTopicTest` only, could you please move 
them into `DeleteTopicTest`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-11-05 Thread via GitHub


TaiJuWu commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1828924456


##
test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java:
##
@@ -225,4 +235,56 @@ default void waitAcls(AclBindingFilter filter, 
Collection en
 }
 }
 
+default  void verifyTopicDeletion(String topic, int 
numPartitions) throws Exception {
+verifyTopicDeletion(topic, numPartitions, brokers().values());
+}
+
+default  void verifyTopicDeletion(String topic, int 
numPartitions, Collection brokers) throws Exception {
+List topicPartitions = IntStream.range(0, 
numPartitions)
+.mapToObj(partition -> new TopicPartition(topic, partition))
+.collect(Collectors.toList());
+
+// Ensure that the topic-partition has been deleted from all brokers' 
replica managers
+TestUtils.waitForCondition(() -> brokers.stream().allMatch(broker ->

Review Comment:
   Since `clusterInstance` support shutdown broker, using `aliveBroker` is 
better solution.
   You can refer https://github.com/apache/kafka/pull/17085



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-11-04 Thread via GitHub


chia7712 commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1828253736


##
test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java:
##
@@ -225,4 +235,56 @@ default void waitAcls(AclBindingFilter filter, 
Collection en
 }
 }
 
+default  void verifyTopicDeletion(String topic, int 
numPartitions) throws Exception {
+verifyTopicDeletion(topic, numPartitions, brokers().values());
+}
+
+default  void verifyTopicDeletion(String topic, int 
numPartitions, Collection brokers) throws Exception {

Review Comment:
   Could you please rename it to `waitForTopicDeletion`? 
   
   Additionally, could you please don't expose `KafkaBroker`? we will cleanup 
all inner interface later and that should not be used by tests anymore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-10-31 Thread via GitHub


frankvicky commented on PR #17579:
URL: https://github.com/apache/kafka/pull/17579#issuecomment-2449604150

   The fail test is handling by #17645


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-10-31 Thread via GitHub


frankvicky commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1824104916


##
core/src/test/java/kafka/admin/DeleteTopicTest.java:
##
@@ -0,0 +1,399 @@
+/*
+ * 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.admin;
+
+import kafka.log.UnifiedLog;
+import kafka.server.KafkaBroker;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.TopicDeletionDisabledException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+import 
org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestDefaults;
+import org.apache.kafka.common.test.api.ClusterTestExtensions;
+import org.apache.kafka.common.test.api.Type;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.metadata.BrokerState;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.common.RequestLocal;
+import org.apache.kafka.server.config.ServerConfigs;
+import org.apache.kafka.storage.internals.log.AppendOrigin;
+import org.apache.kafka.storage.internals.log.VerificationGuard;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import scala.jdk.javaapi.OptionConverters;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(types = {Type.KRAFT},
+brokers = 3,
+serverProperties = {
+@ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+@ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"),
+@ClusterConfigProperty(key = "log.initial.task.delay.ms", value = 
"100"),
+@ClusterConfigProperty(key = "log.segment.delete.delay.ms", value = 
"1000")
+})
+public class DeleteTopicTest {
+private static final String DEFAULT_TOPIC = "topic";
+private final Map> expectedReplicaAssignment = 
Map.of(0, List.of(0, 1, 2));
+
+@ClusterTest
+public void testDeleteTopicWithAllAliveReplicas(ClusterInstance cluster) 
throws Exception {
+try (Admin admin = cluster.createAdminClient()) {
+admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, 
expectedReplicaAssignment))).all().get();
+admin.deleteTopics(List.of(DEFAULT_TOPIC)).all().get();
+cluster.verifyTopicDeletion(DEFAULT_TOPIC, 1);
+}
+}
+
+@ClusterTest
+public void testResumeDeleteTopicWithRecoveredFollower(ClusterInstance 
cluster) throws Exception {
+try (Admin admin = cluster.createAdminClient()) {
+admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, 
expectedReplicaAssignment))).all().get();
+TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 
0);
+int leaderId = waitUtilLeaderIsKnown(cluster.brokers(), 
topicPartition);
+KafkaBroker follower = findFollower(cluster.brokers().values(), 
leaderId);
+
+// shutdown one follower replica
+follower.shutdown();
+admin.deleteTo

Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-10-31 Thread via GitHub


TaiJuWu commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1824099326


##
core/src/test/java/kafka/admin/DeleteTopicTest.java:
##
@@ -0,0 +1,399 @@
+/*
+ * 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.admin;
+
+import kafka.log.UnifiedLog;
+import kafka.server.KafkaBroker;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.TopicDeletionDisabledException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+import 
org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestDefaults;
+import org.apache.kafka.common.test.api.ClusterTestExtensions;
+import org.apache.kafka.common.test.api.Type;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.metadata.BrokerState;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.common.RequestLocal;
+import org.apache.kafka.server.config.ServerConfigs;
+import org.apache.kafka.storage.internals.log.AppendOrigin;
+import org.apache.kafka.storage.internals.log.VerificationGuard;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import scala.jdk.javaapi.OptionConverters;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(types = {Type.KRAFT},
+brokers = 3,
+serverProperties = {
+@ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+@ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"),
+@ClusterConfigProperty(key = "log.initial.task.delay.ms", value = 
"100"),
+@ClusterConfigProperty(key = "log.segment.delete.delay.ms", value = 
"1000")
+})
+public class DeleteTopicTest {
+private static final String DEFAULT_TOPIC = "topic";
+private final Map> expectedReplicaAssignment = 
Map.of(0, List.of(0, 1, 2));
+
+@ClusterTest
+public void testDeleteTopicWithAllAliveReplicas(ClusterInstance cluster) 
throws Exception {
+try (Admin admin = cluster.createAdminClient()) {
+admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, 
expectedReplicaAssignment))).all().get();
+admin.deleteTopics(List.of(DEFAULT_TOPIC)).all().get();
+cluster.verifyTopicDeletion(DEFAULT_TOPIC, 1);
+}
+}
+
+@ClusterTest
+public void testResumeDeleteTopicWithRecoveredFollower(ClusterInstance 
cluster) throws Exception {
+try (Admin admin = cluster.createAdminClient()) {
+admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, 
expectedReplicaAssignment))).all().get();
+TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 
0);
+int leaderId = waitUtilLeaderIsKnown(cluster.brokers(), 
topicPartition);
+KafkaBroker follower = findFollower(cluster.brokers().values(), 
leaderId);
+
+// shutdown one follower replica
+follower.shutdown();
+admin.deleteTopic

Re: [PR] KAFKA-17837: Rewrite DeleteTopicTest [kafka]

2024-10-26 Thread via GitHub


chia7712 commented on code in PR #17579:
URL: https://github.com/apache/kafka/pull/17579#discussion_r1817940100


##
test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java:
##
@@ -125,4 +155,210 @@ static void retryOnExceptionWithTimeout(final Runnable 
runnable) throws Interrup
 Thread.sleep(DEFAULT_POLL_INTERVAL_MS);
 }
 }
+
+/**
+ * Wrap a single record log buffer.
+ */
+public static MemoryRecords singletonRecords(byte[] value,
+ byte[] key,
+ Compression codec,
+ long timestamp,
+ byte magicValue) {
+return records(Collections.singletonList(new SimpleRecord(timestamp, 
key, value)), magicValue, codec);
+}
+
+public static MemoryRecords singletonRecords(byte[] value, byte[] key) {
+return singletonRecords(value, key, Compression.NONE, 
RecordBatch.NO_TIMESTAMP, RecordBatch.CURRENT_MAGIC_VALUE);
+}
+
+public static MemoryRecords records(List records,
+byte magicValue,
+Compression codec,
+long producerId,
+short producerEpoch,
+int sequence,
+long baseOffset,
+int partitionLeaderEpoch) {
+int sizeInBytes = DefaultRecordBatch.sizeInBytes(records);
+ByteBuffer buffer = ByteBuffer.allocate(sizeInBytes);
+
+try (MemoryRecordsBuilder builder = MemoryRecords.builder(
+buffer,
+magicValue,
+codec,
+TimestampType.CREATE_TIME,
+baseOffset,
+System.currentTimeMillis(),
+producerId,
+producerEpoch,
+sequence,
+false,
+partitionLeaderEpoch
+)) {
+records.forEach(builder::append);
+return builder.build();
+}
+}
+
+public static MemoryRecords records(List records, byte 
magicValue, Compression codec) {
+return records(records,
+magicValue,
+codec,
+RecordBatch.NO_PRODUCER_ID,
+RecordBatch.NO_PRODUCER_EPOCH,
+RecordBatch.NO_SEQUENCE,
+0L,
+RecordBatch.NO_PARTITION_LEADER_EPOCH);
+}
+
+public static  void verifyTopicDeletion(String 
topic,
+   int 
numPartitions,
+   
Collection brokers) throws Exception {
+List topicPartitions = IntStream.range(0, 
numPartitions)
+.mapToObj(partition -> new TopicPartition(topic, partition))
+.collect(Collectors.toList());
+
+// Ensure that the topic-partition has been deleted from all brokers' 
replica managers
+TestUtils.waitForCondition(() -> brokers.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(() -> brokers.stream().allMatch(broker ->
+topicPartitions.stream().allMatch(tp -> 
broker.logManager().getLog(tp, false).isEmpty())),
+"Replica logs not deleted after delete topic is complete");
+
+// Ensure that the topic is removed from all cleaner offsets
+TestUtils.waitForCondition(() -> brokers.stream().allMatch(broker ->
+topicPartitions.stream().allMatch(tp -> {
+List liveLogDirs = 
CollectionConverters.asJava(broker.logManager().liveLogDirs());
+return liveLogDirs.stream().allMatch(logDir -> {
+OffsetCheckpointFile checkpointFile;
+try {
+checkpointFile = new OffsetCheckpointFile(new 
File(logDir, "cleaner-offset-checkpoint"), null);
+} catch (IOException e) {
+throw new RuntimeException(e);
+}
+return !checkpointFile.read().containsKey(tp);
+});
+})),
+"Cleaner offset for deleted partition should have been removed");
+
+// Ensure that the topic directories are soft-deleted
+TestUtils.waitForCondition(() -> brokers.stream().allMatch(broker ->
+
CollectionConverters.asJava(broker.config().logDirs()).stream().allMatch(logDir 
->
+topicPartitions.stream().noneMatch(tp ->
+new File(