[jira] [Commented] (KAFKA-5418) ZkUtils.getAllPartitions() may fail if a topic is marked for deletion

2017-06-13 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16047807#comment-16047807
 ] 

ASF GitHub Bot commented on KAFKA-5418:
---

Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3295


> ZkUtils.getAllPartitions() may fail if a topic is marked for deletion
> -
>
> Key: KAFKA-5418
> URL: https://issues.apache.org/jira/browse/KAFKA-5418
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.9.0.1, 0.10.2.1
>Reporter: Edoardo Comar
>Assignee: Mickael Maison
> Fix For: 0.11.0.0
>
>
> Running {{ZkUtils.getAllPartitions()}} on a cluster which had a topic stuck 
> in the 'marked for deletion' state 
> so it was a child of {{/brokers/topics}}
> but it had no children, i.e. the path {{/brokers/topics/thistopic/partitions}}
> did not exist, throws a ZkNoNodeException while iterating:
> {noformat}
> rg.I0Itec.zkclient.exception.ZkNoNodeException: 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /brokers/topics/xyzblahfoo/partitions
>   at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
>   at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:995)
>   at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:675)
>   at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:671)
>   at kafka.utils.ZkUtils.getChildren(ZkUtils.scala:537)
>   at 
> kafka.utils.ZkUtils$$anonfun$getAllPartitions$1.apply(ZkUtils.scala:817)
>   at 
> kafka.utils.ZkUtils$$anonfun$getAllPartitions$1.apply(ZkUtils.scala:816)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>   at scala.collection.Iterator$class.foreach(Iterator.scala:742)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
>   at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>   at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>   at scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>   at kafka.utils.ZkUtils.getAllPartitions(ZkUtils.scala:816)
> ...
>   at java.lang.Thread.run(Thread.java:809)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: 
> KeeperErrorCode = NoNode for /brokers/topics/xyzblahfoo/partitions
>   at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
>   at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>   at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1472)
>   at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1500)
>   at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:114)
>   at org.I0Itec.zkclient.ZkClient$4.call(ZkClient.java:678)
>   at org.I0Itec.zkclient.ZkClient$4.call(ZkClient.java:675)
>   at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:985)
>   ... 
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-5418) ZkUtils.getAllPartitions() may fail if a topic is marked for deletion

2017-06-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16045678#comment-16045678
 ] 

ASF GitHub Bot commented on KAFKA-5418:
---

GitHub user mimaison opened a pull request:

https://github.com/apache/kafka/pull/3295

KAFKA-5418: ZkUtils.getAllPartitions() may fail if a topic is marked for 
deletion

Skip topics that don't have any partitions in zkUtils.getAllPartitions()

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/mimaison/kafka KAFKA-5418

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3295.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3295


commit 0aeca093c2acf47b8d4fa01b68eaef79f625e091
Author: Mickael Maison 
Date:   2017-06-10T20:10:40Z

KAFKA-5418: ZkUtils.getAllPartitions() may fail if a topic is marked for 
deletion




> ZkUtils.getAllPartitions() may fail if a topic is marked for deletion
> -
>
> Key: KAFKA-5418
> URL: https://issues.apache.org/jira/browse/KAFKA-5418
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.9.0.1, 0.10.2.1
>Reporter: Edoardo Comar
>
> Running {{ZkUtils.getAllPartitions()}} on a cluster which had a topic stuck 
> in the 'marked for deletion' state 
> so it was a child of {{/brokers/topics}}
> but it had no children, i.e. the path {{/brokers/topics/thistopic/partitions}}
> did not exist, throws a ZkNoNodeException while iterating:
> {noformat}
> rg.I0Itec.zkclient.exception.ZkNoNodeException: 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /brokers/topics/xyzblahfoo/partitions
>   at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
>   at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:995)
>   at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:675)
>   at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:671)
>   at kafka.utils.ZkUtils.getChildren(ZkUtils.scala:537)
>   at 
> kafka.utils.ZkUtils$$anonfun$getAllPartitions$1.apply(ZkUtils.scala:817)
>   at 
> kafka.utils.ZkUtils$$anonfun$getAllPartitions$1.apply(ZkUtils.scala:816)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>   at scala.collection.Iterator$class.foreach(Iterator.scala:742)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
>   at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>   at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>   at scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>   at kafka.utils.ZkUtils.getAllPartitions(ZkUtils.scala:816)
> ...
>   at java.lang.Thread.run(Thread.java:809)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: 
> KeeperErrorCode = NoNode for /brokers/topics/xyzblahfoo/partitions
>   at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
>   at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>   at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1472)
>   at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1500)
>   at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:114)
>   at org.I0Itec.zkclient.ZkClient$4.call(ZkClient.java:678)
>   at org.I0Itec.zkclient.ZkClient$4.call(ZkClient.java:675)
>   at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:985)
>   ... 
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (KAFKA-5418) ZkUtils.getAllPartitions() may fail if a topic is marked for deletion

2017-06-09 Thread huxihx (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16045321#comment-16045321
 ] 

huxihx commented on KAFKA-5418:
---

Is it a duplicate of 
[KAFKA-1019|https://issues.apache.org/jira/browse/KAFKA-1019]?

> ZkUtils.getAllPartitions() may fail if a topic is marked for deletion
> -
>
> Key: KAFKA-5418
> URL: https://issues.apache.org/jira/browse/KAFKA-5418
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.9.0.1, 0.10.2.1
>Reporter: Edoardo Comar
>
> Running {{ZkUtils.getAllPartitions()}} on a cluster which had a topic stuck 
> in the 'marked for deletion' state 
> so it was a child of {{/brokers/topics}}
> but it had no children, i.e. the path {{/brokers/topics/thistopic/partitions}}
> did not exist, throws a ZkNoNodeException while iterating:
> {noformat}
> rg.I0Itec.zkclient.exception.ZkNoNodeException: 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /brokers/topics/xyzblahfoo/partitions
>   at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
>   at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:995)
>   at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:675)
>   at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:671)
>   at kafka.utils.ZkUtils.getChildren(ZkUtils.scala:537)
>   at 
> kafka.utils.ZkUtils$$anonfun$getAllPartitions$1.apply(ZkUtils.scala:817)
>   at 
> kafka.utils.ZkUtils$$anonfun$getAllPartitions$1.apply(ZkUtils.scala:816)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>   at scala.collection.Iterator$class.foreach(Iterator.scala:742)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
>   at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>   at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>   at scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>   at kafka.utils.ZkUtils.getAllPartitions(ZkUtils.scala:816)
> ...
>   at java.lang.Thread.run(Thread.java:809)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: 
> KeeperErrorCode = NoNode for /brokers/topics/xyzblahfoo/partitions
>   at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
>   at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>   at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1472)
>   at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1500)
>   at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:114)
>   at org.I0Itec.zkclient.ZkClient$4.call(ZkClient.java:678)
>   at org.I0Itec.zkclient.ZkClient$4.call(ZkClient.java:675)
>   at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:985)
>   ... 
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)