Jun,
      I made an attempt at fixing that issue as part of this JIRA
      https://issues.apache.org/jira/browse/KAFKA-1507 . 
As Jay pointed out there should be admin api if there is more info on
this api I am interested in adding/fixing this issue.
Thanks,
Harsha

On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> Yes, that's the issue. Currently, topics can be auto-created on
> TopicMetadataRequest, which can be issued from both the producer and the
> consumer. To prevent that, you would need to stop the producer and the
> consumer before deleting a topic. We plan to address this issue once we
> have a separate request for creating topics.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Jan 26, 2015 at 7:21 AM, Harsha <ka...@harsha.io> wrote:
> 
> > There could be another case where if you have auto.create.topics.enable
> > to set to true ( its true by default) . Any TopicMetadataRequest can
> > recreate topics. So if you issued a delete topic command and you have
> > producers running or consumers? too which is issuing a
> > TopicMetadataRequest than the topic will be recreated.
> > -Harsha
> >
> > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > cversion did change (incremented by 2) when I issue the delete command.
> > >
> > > From the logs on the conroller broker (also the leader for the topic), it
> > > looks like the delete proceeds, and then the topic gets recreated
> > > immediately (highlighted in yellow). It appears maybe it’s due to a
> > > consumer client app trying to consume the topic. Also, the consumer is
> > > not
> > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> > > problem?
> > >
> > >
> > > 2015-01-26 07:02:14,281  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > mytopic
> > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > > deletion callback for mytopic
> > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > Partition deletion callback for [mytopic,0]
> > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to OfflineReplica for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to ReplicaDeletionStarted for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > log.OffsetIndex - Deleting index
> > > /mypath/mytopic-0/00000000000000000000.index
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7] log.LogManager
> > > - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> > > 2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> > > [Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,314  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Deletion for replicas 7 for partition
> > > [mytopic,0] of topic mytopic in progress
> > > 2015-01-26 07:02:14,316  INFO [Controller-6-to-broker-7-send-thread]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> > > [Topic=mytopic,Partition=0,Replica=7]
> > > 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to NonExistentReplica for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=6],[Topic=mytopic,Partition=0,Replica=7]
> > > 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> > > controller.PartitionStateMachine - [Partition state machine on
> > > Controller 6]: Invoking state change to OfflinePartition for
> > > partitions [mytopic,0]
> > > 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> > > controller.PartitionStateMachine - [Partition state machine on
> > > Controller 6]: Invoking state change to NonExistentPartition for
> > > partitions [mytopic,0]
> > > 2015-01-26 07:02:14,325  WARN [kafka-request-handler-2]
> > > server.ReplicaManager - [Replica Manager on Broker 6]: Fetch request
> > > with correlation id 2619442 from client ReplicaFetcherThread-0-6 on
> > > partition [mytopic,0] failed due to Partition [mytopic,0] doesn't
> > > exist on 6
> > > 2015-01-26 07:02:14,371  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Deletion of topic mytopic successfully
> > > completed
> > > 2015-01-26 07:02:14,376  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.PartitionStateMachine$TopicChangeListener -
> > > [TopicChangeListener on Controller 6]: New topics: [Set()], deleted
> > > topics: [Set()], new partition replica assignment [Map()]
> > > 2015-01-26 07:02:14,403  WARN [request-expiration-task]
> > > server.ReplicaManager - [Replica Manager on Broker 6]: Fetch request
> > > with correlation id 2054626 from client
> > > myconsumer-1415333962444-cc78e67e-0-6 on partition [mytopic,0] failed
> > > due to Partition [mytopic,0] doesn't exist on 6
> > > 2015-01-26 07:02:14,446  INFO [kafka-request-handler-5]
> > > admin.AdminUtils$ - Topic creation
> > > {"version":1,"partitions":{"0":[7,6]}}
> > > 2015-01-26 07:02:14,457  INFO [kafka-request-handler-5]
> > > server.KafkaApis - [KafkaApi-6] Auto creation of topic mytopic with 1
> > > partitions and replication factor 2 is successful!
> > > 2015-01-26 07:02:14,458  INFO [kafka-network-thread-27330-2]
> > > network.Processor - Closing socket connection to /10.1.10.4.
> > > 2015-01-26 07:02:14,466  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.PartitionStateMachine$TopicChangeListener -
> > > [TopicChangeListener on Controller 6]: New topics: [Set(mytopic)],
> > > deleted topics: [Set()], new partition replica assignment
> > > [Map([mytopic,0] -> List(7, 6))]
> > > 2015-01-26 07:02:14,466  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.KafkaController - [Controller 6]: New topic creation
> > > callback for [mytopic,0]
> > > 2015-01-26 07:02:14,466  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.KafkaController - [Controller 6]: New partition creation
> > > callback for [mytopic,0]
> > > 2015-01-26 07:02:14,466  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.PartitionStateMachine - [Partition state machine on
> > > Controller 6]: Invoking state change to NewPartition for partitions
> > > [mytopic,0]
> > > 2015-01-26 07:02:14,467  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to NewReplica for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,468  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.PartitionStateMachine - [Partition state machine on
> > > Controller 6]: Invoking state change to OnlinePartition for partitions
> > > [mytopic,0]
> > > 2015-01-26 07:02:14,530  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to OnlineReplica for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,532  INFO [kafka-request-handler-5] log.Log -
> > > Completed load of log mytopic-0 with log end offset 0
> > > 2015-01-26 07:02:14,533  INFO [kafka-request-handler-5] log.LogManager
> > > - Created log for partition [mytopic,0] in /mypath with properties
> > > {segment.index.bytes -> 10485760, file.delete.delay.ms -> 60000,
> > > segment.bytes -> 1073741824, flush.ms -> 9223372036854775807,
> > > delete.retention.ms -> 86400000, index.interval.bytes -> 4096,
> > > retention.bytes -> 50000000000, min.insync.replicas -> 1,
> > > cleanup.policy -> delete, unclean.leader.election.enable -> true,
> > > segment.ms -> 604800000, max.message.bytes -> 1000012, flush.messages
> > > -> 9223372036854775807, min.cleanable.dirty.ratio -> 0.5, retention.ms
> > > -> 86400000, segment.jitter.ms -> 0}.
> > > 2015-01-26 07:02:14,540  INFO [kafka-request-handler-5]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,540  INFO [kafka-request-handler-5] log.Log -
> > > Truncating log mytopic-0 to offset 0.
> > > 2015-01-26 07:02:14,547  INFO [kafka-request-handler-5]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > Added fetcher for partitions List([[mytopic,0], initOffset 0 to broker
> > > id:7,host:mybroker:54321] )
> > >
> > > Here’s the log in the other broker with a replica of the topic.
> > >
> > > 2015-01-26 07:02:14,312  INFO [kafka-request-handler-7]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 7]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-2]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 7]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-2]
> > > log.OffsetIndex - Deleting index
> > > /mypath/mytopic-0/00000000000000000000.index
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-2] log.LogManager
> > > - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> > > 2015-01-26 07:02:14,530  INFO [kafka-request-handler-0]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 7]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,532  INFO [kafka-request-handler-0] log.Log -
> > > Completed load of log mytopic-0 with log end offset 0
> > > 2015-01-26 07:02:14,533  INFO [kafka-request-handler-0] log.LogManager
> > > - Created log for partition [mytopic,0] in /mypath with properties
> > > {segment.index.bytes -> 10485760, file.delete.delay.ms -> 60000,
> > > segment.bytes -> 1073741824, flush.ms -> 9223372036854775807,
> > > delete.retention.ms -> 86400000, index.interval.bytes -> 4096,
> > > retention.bytes -> 50000000000, min.insync.replicas -> 1,
> > > cleanup.policy -> delete, unclean.leader.election.enable -> true,
> > > segment.ms -> 604800000, max.message.bytes -> 1000012, flush.messages
> > > -> 9223372036854775807, min.cleanable.dirty.ratio -> 0.5, retention.ms
> > > -> 86400000, segment.jitter.ms -> 0}.
> > >
> > > On Mon, Jan 26, 2015 at 1:26 AM, Jun Rao <j...@confluent.io> wrote:
> > >
> > > Could you do a get on /admin/deleted_topics and see if its cversion
> > > changes
> > > > after issuing the delete topic command? This will tell us whether the
> > > > path /admin/deleted_topics//mytopic
> > > > is actually created or not. Anything in the controller log that
> > indicates a
> > > > delete topic event is fired?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Sun, Jan 25, 2015 at 9:55 PM, Jason Rosenberg <j...@squareup.com>
> > wrote:
> > > >
> > > > > yes
> > > > >
> > > > > On Mon, Jan 26, 2015 at 12:18 AM, Jun Rao <j...@confluent.io> wrote:
> > > > >
> > > > > > Do you have delete.topic.enable turned on in all brokers?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Sun, Jan 25, 2015 at 7:56 PM, Jason Rosenberg <j...@squareup.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > So far, I have been unable to get delete topic to work, with
> > release
> > > > > > > candidate 2 for 0.8.2.
> > > > > > >
> > > > > > > It worked ok when I ran it in the debugger locally, on a single
> > node
> > > > > > > instance. But when I run it in our staging environment, it is not
> > > > > > > successfully even marking the topic for delete, for some reason.
> > > > > > >
> > > > > > > I am setting delete.topic.enable to true (and see the
> > confirmation of
> > > > > > this
> > > > > > > in the startup logs, e.g.):
> > > > > > >
> > > > > > > INFO [main] utils.VerifiableProperties - Property
> > delete.topic.enable
> > > > > > > is overridden to true
> > > > > > >
> > > > > > > I run this command:
> > > > > > >
> > > > > > > java -cp app.jar kafka.admin.TopicCommand --zookeeper
> > > > > > > myzkconnect:12345/mynamespace --delete --topic mytopic
> > > > > > >
> > > > > > > log4j:WARN No appenders could be found for logger
> > > > > > > (org.I0Itec.zkclient.ZkConnection).
> > > > > > > log4j:WARN Please initialize the log4j system properly.
> > > > > > > log4j:WARN No appenders could be found for logger
> > > > > > > (org.I0Itec.zkclient.ZkEventThread).
> > > > > > > log4j:WARN See
> > http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > > > > > for more info.
> > > > > > > log4j:WARN Please initialize the log4j system properly.
> > > > > > > log4j:WARN See
> > http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > > > > > for more info.
> > > > > > > Topic mytopic is marked for deletion.
> > > > > > > Note: This will have no impact if delete.topic.enable is not set
> > to
> > > > > true.
> > > > > > >
> > > > > > > I then do a —list which should at least show the topic marked for
> > > > > > deletion:
> > > > > > >
> > > > > > > java -cp app.jar kafka.admin.TopicCommand --zookeeper
> > > > > > > myzkconnect:12345/mynamespace --list --topic mytopic
> > > > > > >
> > > > > > > log4j:WARN No appenders could be found for logger
> > > > > > > (org.I0Itec.zkclient.ZkConnection).
> > > > > > > log4j:WARN No appenders could be found for logger
> > > > > > > (org.I0Itec.zkclient.ZkEventThread).
> > > > > > > log4j:WARN Please initialize the log4j system properly.
> > > > > > > log4j:WARN Please initialize the log4j system properly.
> > > > > > > log4j:WARN See
> > http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > > > > > for more info.
> > > > > > > log4j:WARN See
> > http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > > > > > for more info.
> > > > > > > mytopic
> > > > > > >
> > > > > > > Note, it doesn’t list it as ‘marked for deletion’. Furthermore,
> > > > waiting
> > > > > > > multiple hours still doesn’t result in the topic being deleted.
> > > > > > >
> > > > > > > The topic has 1 partition, 2 replicas, and no data stored.
> > > > > > >
> > > > > > > In zookeeper, the /admin/deleted_topics/ path is empty.
> > > > > > >
> > > > > > > The zookeeper code looks pretty straightforward, but for some
> > reason
> > > > is
> > > > > > not
> > > > > > > writing the deleted_topics path.  We are running zookeeper 3.4.6.
> > > > > > >
> > > > > > > Thoughts?
> > > > > > >
> > > > > > > Jason
> > > > > > > ​
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > ​
> >

Reply via email to