kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Joel Koshy <jjkosh...@gmail.com>
Subject Re: Issue with topic deletion
Date Thu, 05 Feb 2015 02:14:42 GMT
I took a look at your logs. I agree with Harsh that the logs seem
truncated. The basic issue though is that you have session expirations
and controller failover. Broker 49554 was the controller and hosted
some partition(s) of LAX1-GRIFFIN-r13-1423001701601. After controller
failover the new controller marks it as ineligible for deletion since
49554 is considered down (until it re-registers in zookeeper) and is
relected as the leader - however I don't see those logs.

Any idea why you have session expirations? This is typically due to GC
and/or flaky network. Regardless, we should be handling that scenario
as well. However, your logs seem incomplete. Can you redo this and
perhaps keep the set up running a little longer and send over those
logs?

Thanks,

Joel

On Wed, Feb 04, 2015 at 01:00:46PM -0800, Sumit Rangwala wrote:
> >
> >
> >> I have since stopped the container so I cannot say if
> > LAX1-GRIFFIN-r45-1423000088317 was one of the topic in "marked for
> > deletion" forever.  However, there were many topics (at least 10 of them)
> > that were perennially in "marked for deletion" state.
> >
> >
> I have the setup to recreate the issue in case the logs are not sufficient.
> 
> 
> Sumit
> 
> 
> 
> > Sumit
> >
> >
> >
> >
> >
> >> -Harsha
> >>
> >> On Tue, Feb 3, 2015, at 09:19 PM, Harsha wrote:
> >> > you are probably handling it but there is a case where you call
> >> > deleteTopic and kafka goes through delete topic process but your
> >> > consumer is running probably made a TopicMetadataRequest for the same
> >> > topic which can re-create the topic with the default num.partitions and
> >> > replication.factor.  Did you try stopping the consumer first and issue
> >> > the topic delete.
> >> > -Harsha
> >> >
> >> > On Tue, Feb 3, 2015, at 08:37 PM, Sumit Rangwala wrote:
> >> > > On Tue, Feb 3, 2015 at 6:48 PM, Harsha <kafka@harsha.io> wrote:
> >> > >
> >> > > > Sumit,
> >> > > >        lets say you are deleting a older topic "test1" do you
have
> >> any
> >> > > >        consumers running simultaneously for the topic "test1"
 while
> >> > > >        deletion of topic going on.
> >> > > >
> >> > >
> >> > > Yes it is the case. However, after a small period of time (say few
> >> > > minutes)
> >> > > there won't be any consumer running for the deleted topic.
> >> > >
> >> > >
> >> > > Sumit
> >> > >
> >> > >
> >> > >
> >> > >
> >> > > > -Harsha
> >> > > >
> >> > > > On Tue, Feb 3, 2015, at 06:17 PM, Joel Koshy wrote:
> >> > > > > Thanks for the logs - will take a look tomorrow unless someone
> >> else
> >> > > > > gets a chance to get to it today.
> >> > > > >
> >> > > > > Joel
> >> > > > >
> >> > > > > On Tue, Feb 03, 2015 at 04:11:57PM -0800, Sumit Rangwala
wrote:
> >> > > > > > On Tue, Feb 3, 2015 at 3:37 PM, Joel Koshy <jjkoshy.w@gmail.com
> >> >
> >> > > > wrote:
> >> > > > > >
> >> > > > > > > Hey Sumit,
> >> > > > > > >
> >> > > > > > > I thought you would be providing the actual steps
to
> >> reproduce :)
> >> > > > > > >
> >> > > > > >
> >> > > > > > I want to but some proprietary code prevents me to
do it.
> >> > > > > >
> >> > > > > >
> >> > > > > > > Nevertheless, can you get all the relevant logs:
state change
> >> logs
> >> > > > and
> >> > > > > > > controller logs at the very least and if possible
server logs
> >> and
> >> > > > send
> >> > > > > > > those over?
> >> > > > > > >
> >> > > > > >
> >> > > > > > Here are all the logs you requested (there are three
brokers in
> >> my
> >> > > > setup
> >> > > > > > k1, k2, k3): http://d.pr/f/1kprY/2quHBRRT (Gmail has
issue
> >> with the
> >> > > > file)
> >> > > > > >
> >> > > > > >
> >> > > > > > Sumit
> >> > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > > > >
> >> > > > > > > Joel
> >> > > > > > >
> >> > > > > > > On Tue, Feb 03, 2015 at 03:27:43PM -0800, Sumit
Rangwala
> >> wrote:
> >> > > > > > > > In my setup kafka brokers are set for auto
topic creation.
> >> In the
> >> > > > > > > scenario
> >> > > > > > > > below a node informs other nodes (currently
5 in total)
> >> about  a
> >> > > > number
> >> > > > > > > of
> >> > > > > > > > new (non-existent) topics, and  all the nodes
almost
> >> > > > simultaneously open
> >> > > > > > > a
> >> > > > > > > > consumer for each of those topics. Sometime
later another
> >> node
> >> > > > informs
> >> > > > > > > all
> >> > > > > > > > other nodes of a new list of topics and each
node, if they
> >> find
> >> > > > that an
> >> > > > > > > > older topic exists in kafka, goes ahead and
deletes the
> >> older
> >> > > > topic.
> >> > > > > > > What I
> >> > > > > > > > have found is that many of the topics stay
in the "marked
> >> for
> >> > > > deletion"
> >> > > > > > > > state forever.
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > I get the list of topics using
> >> ZkUtils.getAllTopics(zkClient) and
> >> > > > delete
> >> > > > > > > > topics using AdminUtils.deleteTopic(zkClient,
topic). Since
> >> many
> >> > > > nodes
> >> > > > > > > > might try to delete the same topic at the
same time I do
> >> > > > > > > > see ZkNodeExistsException while deleting
the topic, which I
> >> catch
> >> > > > an
> >> > > > > > > > ignore. (e.g.,
> >> > > > org.apache.zookeeper.KeeperException$NodeExistsException:
> >> > > > > > > > KeeperErrorCode = NodeExists for
> >> > > > > > > > /admin/delete_topics/LAX1-GRIFFIN-r13-1423001701601)
> >> > > > > > > >
> >> > > > > > > > # State of one deleted topic on kafka brokers:
> >> > > > > > > > Topic:LAX1-GRIFFIN-r13-1423001701601 PartitionCount:8
> >> > > > ReplicationFactor:1
> >> > > > > > > > Configs:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
0 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49558 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
1 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49554 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
2 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49557 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
3 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49558 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
4 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49554 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
5 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49557 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
6 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49558 Isr:
> >> > > > > > > > Topic: LAX1-GRIFFIN-r13-1423001701601 Partition:
7 Leader:
> >> -1
> >> > > > Replicas:
> >> > > > > > > > 49554 Isr:
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > # Controller log says
> >> > > > > > > >
> >> > > > > > > > [2015-02-03 22:59:03,399] INFO [delete-topics-thread-49554],
> >> > > > Deletion for
> >> > > > > > > > replicas 49557,49554,49558 for partition
> >> > > > > > > >
> >> > > > > > >
> >> > > >
> >> [LAX1-GRIFFIN-r13-1423001701601,0],[LAX1-GRIFFIN-r13-1423001701601,6],[LAX1-GRIFFIN-r13-1423001701601,5],[LAX1-GRIFFIN-r13-1423001701601,3],[LAX1-GRIFFIN-r13-1423001701601,7],[LAX1-GRIFFIN-r13-1423001701601,1],[LAX1-GRIFFIN-r13-1423001701601,4],[LAX1-GRIFFIN-r13-1423001701601,2]
> >> > > > > > > > of topic LAX1-GRIFFIN-r13-1423001701601 in
progress
> >> > > > > > > > (kafka.controller.TopicDeletionManager$DeleteTopicsThread)
> >> > > > > > > >
> >> > > > > > > > current time: Tue Feb  3 23:20:58 UTC 2015
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > Since I don't know the delete topic algorithm,
I am not
> >> sure why
> >> > > > sure
> >> > > > > > > these
> >> > > > > > > > topics are not garbage collected. I do have
the complete
> >> setup
> >> > > > running in
> >> > > > > > > > docker right now on my local box so please
let me know if
> >> any more
> >> > > > info
> >> > > > > > > is
> >> > > > > > > > required to troubleshoot this issue.
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > Furthermore, does ZkUtils.getAllTopics(zkClient)
return
> >> "marked for
> >> > > > > > > > deletion" topic as well. If so, is there
a easy way to get
> >> a list
> >> > > > of
> >> > > > > > > active
> >> > > > > > > > topics (other than looking at all the topics
in
> >> > > > /admin/delete_topics/ and
> >> > > > > > > > taking a set difference with the topics returned
> >> > > > > > > > by ZkUtils.getAllTopics(zkClient) )
> >> > > > > > > >
> >> > > > > > > > Sumit
> >> > > > > > > > (More setup info below)
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > Setup
> >> > > > > > > > --------
> >> > > > > > > > Zookeeper: 3.4.6
> >> > > > > > > > Kafka broker: 0.8.2-beta
> >> > > > > > > > Kafka clients: 0.8.2-beta
> >> > > > > > > >
> >> > > > > > > > # Kafka boker settings (all other settings
are default
> >> 0.8.2-beta
> >> > > > > > > settings)
> >> > > > > > > > kafka.controlled.shutdown.enable: 'FALSE'
> >> > > > > > > > kafka.auto.create.topics.enable: 'TRUE'
> >> > > > > > > > kafka.num.partitions: 8
> >> > > > > > > > kafka.default.replication.factor: 1
> >> > > > > > > > kafka.rebalance.backoff.ms: 3000
> >> > > > > > > > kafka.rebalance.max.retries: 10
> >> > > > > > > > kafka.log.retention.minutes: 1200
> >> > > > > > > > kafka.delete.topic.enable: 'TRUE'
> >> > > > > > >
> >> > > > > > >
> >> > > > >
> >> > > >
> >>
> >
> >


Mime
View raw message