@@ -29,7 +29,8 @@ import scala.util.Random
2929
3030import kafka .admin .AdminUtils
3131import kafka .api .Request
32- import kafka .server .{KafkaConfig , KafkaServer }
32+ import kafka .common .TopicAndPartition
33+ import kafka .server .{KafkaConfig , KafkaServer , OffsetCheckpoint }
3334import kafka .utils .ZkUtils
3435import org .apache .kafka .clients .consumer .KafkaConsumer
3536import org .apache .kafka .clients .producer ._
@@ -171,6 +172,12 @@ class KafkaTestUtils extends Logging {
171172 createTopic(topic, 1 )
172173 }
173174
175+ /** Delete a Kafka topic and wait until it is propagated to the whole cluster */
176+ def deleteTopic (topic : String , partitions : Int ): Unit = {
177+ AdminUtils .deleteTopic(zkUtils, topic)
178+ verifyTopicDeletion(zkUtils, topic, partitions, List (this .server))
179+ }
180+
174181 /** Add new paritions to a Kafka topic */
175182 def addPartitions (topic : String , partitions : Int ): Unit = {
176183 AdminUtils .addPartitions(zkUtils, topic, partitions)
@@ -234,6 +241,7 @@ class KafkaTestUtils extends Logging {
234241 props.put(" zookeeper.connect" , zkAddress)
235242 props.put(" log.flush.interval.messages" , " 1" )
236243 props.put(" replica.socket.timeout.ms" , " 1500" )
244+ props.put(" delete.topic.enable" , " true" )
237245 props
238246 }
239247
@@ -257,6 +265,37 @@ class KafkaTestUtils extends Logging {
257265 props
258266 }
259267
268+ private def verifyTopicDeletion (
269+ zkUtils : ZkUtils ,
270+ topic : String ,
271+ numPartitions : Int ,
272+ servers : Seq [KafkaServer ]) {
273+ import ZkUtils ._
274+ val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition (topic, _))
275+ def isDeleted (): Boolean = {
276+ // wait until admin path for delete topic is deleted, signaling completion of topic deletion
277+ val deletePath = ! zkUtils.pathExists(getDeleteTopicPath(topic))
278+ val topicPath = ! zkUtils.pathExists(getTopicPath(topic))
279+ // ensure that the topic-partition has been deleted from all brokers' replica managers
280+ val replicaManager = servers.forall(server => topicAndPartitions.forall(tp =>
281+ server.replicaManager.getPartition(tp.topic, tp.partition) == None ))
282+ // ensure that logs from all replicas are deleted if delete topic is marked successful
283+ val logManager = servers.forall(server => topicAndPartitions.forall(tp =>
284+ server.getLogManager().getLog(tp).isEmpty))
285+ // ensure that topic is removed from all cleaner offsets
286+ val cleaner = servers.forall(server => topicAndPartitions.forall { tp =>
287+ val checkpoints = server.getLogManager().logDirs.map { logDir =>
288+ new OffsetCheckpoint (new File (logDir, " cleaner-offset-checkpoint" )).read()
289+ }
290+ checkpoints.forall(checkpointsPerLogDir => ! checkpointsPerLogDir.contains(tp))
291+ })
292+ deletePath && topicPath && replicaManager && logManager && cleaner
293+ }
294+ eventually(timeout(10 .seconds)) {
295+ assert(isDeleted, s " $topic not deleted after timeout " )
296+ }
297+ }
298+
260299 private def waitUntilMetadataIsPropagated (topic : String , partition : Int ): Unit = {
261300 def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match {
262301 case Some (partitionState) =>
0 commit comments