Sorry, I do not know if this is related to KAFKA-2937, I let you judge and
decide whether ticket needs to be reopened or new one created.

I'm still missing full picture what was happening but here are more pieces.

1) Topic foo was created (likely with Kafka 0.8.2.x). Topic foo was never
used (no messages published to it ever).
2) Topic bar was created with Kafka 0.9.0.1
3) Deletion for topic foo was requested, and replica deletion got stuck
(see [1] server.log fragment)
4) Deletion for topic bar was requested, one bar replica was on same broker
as replica for foo whose deletion got stuck, and deletion for bar replica
was never even tried
5) Topics got in non-deletable state (see [2] controller.log fragment)

As you can see from [1] replica had one state in cache, while actual in ZK
was different. I'm not sure why /brokers/topics/foo/partitions/0/state did
not

[1] replica server.log fragment
...
[2016-03-04 09:42:03,495] INFO [ReplicaFetcherManager on broker 1] Removed
fetcher for partitions [foo,0] (kafka.server.ReplicaFetcherManager)
[2016-03-04 09:42:03,533] INFO [ReplicaFetcherManager on broker 1] Removed
fetcher for partitions [foo,0] (kafka.server.ReplicaFetcherManager)
[2016-03-04 09:42:13,894] INFO Partition [foo,0] on broker 1: Shrinking ISR
for partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:13,897] WARN Conditional update of path
/brokers/topics/foo/partitions/0/state with data
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]}
and expected version 68 failed due to
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode =
NoNode for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:13,898] INFO Partition [foo,0] on broker 1: Cached
zkVersion [68] not equal to that in zookeeper, skip updating ISR
(kafka.cluster.Partition)
[2016-03-04 09:42:23,894] INFO Partition [foo,0] on broker 1: Shrinking ISR
for partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:23,897] WARN Conditional update of path
/brokers/topics/foo/partitions/0/state with data
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]}
and expected version 68 failed due to
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode =
NoNode for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:23,897] INFO Partition [foo,0] on broker 1: Cached
zkVersion [68] not equal to that in zookeeper, skip updating ISR
(kafka.cluster.Partition)
[2016-03-04 09:42:33,894] INFO Partition [foo,0] on broker 1: Shrinking ISR
for partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:33,897] WARN Conditional update of path
/brokers/topics/foo/partitions/0/state with data
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]}
and expected version 68 failed due to
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode =
NoNode for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:33,897] INFO Partition [foo,0] on broker 1: Cached
zkVersion [68] not equal to that in zookeeper, skip updating ISR
(kafka.cluster.Partition)
...

[2] controller.log fragment
...
[2016-03-04 09:59:06,753] DEBUG [Topic Deletion Manager 1], Deletion failed
for replicas [Topic=bar,Partition=0,Replica=0]. Halting deletion for topics
Set(bar) (kafka.controller.TopicDeletionManager)
[2016-03-04 09:59:06,753] INFO [Replica state machine on controller 1]:
Invoking state change to ReplicaDeletionIneligible for replicas
[Topic=bar,Partition=0,Replica=0] (kafka.controller.ReplicaStateMachine)
[2016-03-04 09:59:06,754] INFO [Topic Deletion Manager 1], Halted deletion
of topics bar (kafka.controller.TopicDeletionManager)
...

On Fri, Mar 11, 2016 at 5:00 PM, Mayuresh Gharat <gharatmayures...@gmail.com
> wrote:

> Hi Stevo,
>
> Let me know if we want to open Kafka-2937 again. I can include the above
> finding in to the patch or you want to create a separate JIra for this.
>
> Thanks,
>
> Mayuresh
>
> On Fri, Mar 11, 2016 at 7:53 AM, Mayuresh Gharat <
> gharatmayures...@gmail.com
> > wrote:
>
> > kafka-2937 is different from this I think. Kafka-2937 deals with the
> > delete topic getting stuck because the LeaderAndISR in ZK was updated by
> a
> > controller and then the controller dies and the new controller gets in to
> > the exception and never completes deleting the topic. The topic existed
> in
> > the cluster and was also marked for delete.
> > The case reported here is that the topic does not exist in cluster but is
> > marked for delete.
> > Am I right in understanding?
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Fri, Mar 11, 2016 at 5:30 AM, Stevo Slavić <ssla...@gmail.com> wrote:
> >
> >> Topic it seems would get deleted but request in ZK to delete topic would
> >> not get cleared even after restarting Kafka cluster.
> >>
> >> I'm still investigating why deletion did not complete in the first place
> >> without restarting any nodes. It seems something smelly happens when
> there
> >> is request to delete more than one topic.
> >>
> >> Anyway, I think I found one potential bug in
> >> ReplicaStateMachine.areAllReplicasForTopicDeleted check which could be
> >> cause for not clearing deletion request from ZK even after restart of
> >> whole
> >> cluster. Line ReplicaStateMachine.scala#L285
> >> <
> >>
> https://github.com/sslavic/kafka/blob/trunk/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala#L285
> >> >
> >>
> >> replicaStatesForTopic.forall(_._2 == ReplicaDeletionSuccessful)
> >>
> >> which is return value of that function/check, probably should better be
> >> checking for
> >>
> >> replicaStatesForTopic.isEmpty || replicaStatesForTopic.forall(_._2 ==
> >> ReplicaDeletionSuccessful)
> >>
> >> I noticed it because in controller logs I found entries like:
> >>
> >> [2016-03-04 13:27:29,115] DEBUG [Replica state machine on controller 1]:
> >> Are all replicas for topic foo deleted Map()
> >> (kafka.controller.ReplicaStateMachine)
> >>
> >> even though normally they look like:
> >>
> >> [2016-03-04 09:33:41,036] DEBUG [Replica state machine on controller 1]:
> >> Are all replicas for topic foo deleted
> >> Map([Topic=foo,Partition=0,Replica=0] -> ReplicaDeletionStarted,
> >> [Topic=foo,Partition=0,Replica=3] -> ReplicaDeletionStarted,
> >> [Topic=foo,Partition=0,Replica=1] -> ReplicaDeletionSuccessful)
> >> (kafka.controller.ReplicaStateMachine)
> >>
> >> Kind regards,
> >> Stevo Slavic.
> >>
> >> On Sun, Mar 6, 2016 at 12:31 AM, Guozhang Wang <wangg...@gmail.com>
> >> wrote:
> >>
> >> > Thanks Stevo,
> >> >
> >> > Feel free to paste your findings in KAFKA-2937, we can re-open that
> >> ticket
> >> > if necessary.
> >> >
> >> > Guozhang
> >> >
> >> > On Fri, Mar 4, 2016 at 4:38 AM, Stevo Slavić <ssla...@gmail.com>
> wrote:
> >> >
> >> > > Hell Apache Kafka community,
> >> > >
> >> > > I'm still investigating an incident; from initial findings topic
> >> deletion
> >> > > doesn't seem to work well still with Kafka 0.9.0.1, likely some edge
> >> case
> >> > > not covered.
> >> > >
> >> > > Before with 0.8.2.x it used to happen that non-lead replica would be
> >> > stuck
> >> > > in topic deletion process, and workaround was just to restart that
> >> node.
> >> > >
> >> > > If I'm not mistaken, that edge case got (or at least is expected to
> >> be)
> >> > > fixed in 0.9.0.1 via KAFKA-2937
> >> > > <https://issues.apache.org/jira/browse/KAFKA-2937>
> >> > >
> >> > > Request to delete topic continued to be there in ZK even after whole
> >> > > cluster restart - topic seemed not to exist, seemed to actually be
> >> > deleted,
> >> > > but request to delete topic would remain. Had to manually delete
> >> request
> >> > > node in ZK.
> >> > >
> >> > > When I have more details, and reproducible use case, will report
> back.
> >> > >
> >> > > Kind regards,
> >> > > Stevo Slavic.
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > -- Guozhang
> >> >
> >>
> >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Reply via email to