[
https://issues.apache.org/jira/browse/KAFKA-3783?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15313063#comment-15313063
]
ASF GitHub Bot commented on KAFKA-3783:
---------------------------------------
GitHub user slaunay opened a pull request:
https://github.com/apache/kafka/pull/1461
KAFKA-3783: Catch proper exception on path delete
- ZkClient is used for conditional path deletion and wraps
`KeeperException.BadVersionException` into `ZkBadVersionException`
- add unit test to `SimpleAclAuthorizerTest` to reproduce the issue and
catch potential future regression
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/slaunay/kafka
bugfix/KAFKA-3783-zk-conditional-delete-path
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/kafka/pull/1461.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 #1461
----
commit d4668ceed50f634fae764a8bb09983a0f8d363d5
Author: Sebastien Launay <[email protected]>
Date: 2016-06-02T20:32:44Z
KAFKA-3783: Catch proper exception on path delete
ZkClient is used for conditional path deletion and wraps
KeeperException.BadVersionException into ZkBadVersionException
----
> Race condition on last ACL removal for a resource fails with a
> ZkBadVersionException
> ------------------------------------------------------------------------------------
>
> Key: KAFKA-3783
> URL: https://issues.apache.org/jira/browse/KAFKA-3783
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 0.9.0.1, 0.10.0.0
> Reporter: Sébastien Launay
> Priority: Minor
>
> When removing the last ACL for a given resource, the znode storing the ACLs
> will get removed.
> The version number of the znode is used for optimistic locking in a loop to
> provide atomic changes across brokers.
> Unfortunately the exception thrown when the operation fails because of a
> different version number is the wrong one
> ({{KeeperException.BadVersionException}} instead of ZkClient
> {{ZkBadVersionException}}) and does not get caught resulting in the
> following stack trace:
> {noformat}
> org.I0Itec.zkclient.exception.ZkBadVersionException:
> org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode =
> BadVersion for /kafka-acl/Topic/e6df8028-f268-408c-814e-d418e943b2fa
> at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:51)
> at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:1000)
> at org.I0Itec.zkclient.ZkClient.delete(ZkClient.java:1047)
> at kafka.utils.ZkUtils.conditionalDeletePath(ZkUtils.scala:522)
> at
> kafka.security.auth.SimpleAclAuthorizer.kafka$security$auth$SimpleAclAuthorizer$$updateResourceAcls(SimpleAclAuthorizer.scala:282)
> at
> kafka.security.auth.SimpleAclAuthorizer$$anonfun$removeAcls$1.apply$mcZ$sp(SimpleAclAuthorizer.scala:187)
> at
> kafka.security.auth.SimpleAclAuthorizer$$anonfun$removeAcls$1.apply(SimpleAclAuthorizer.scala:187)
> at
> kafka.security.auth.SimpleAclAuthorizer$$anonfun$removeAcls$1.apply(SimpleAclAuthorizer.scala:187)
> at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:231)
> at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:239)
> at
> kafka.security.auth.SimpleAclAuthorizer.removeAcls(SimpleAclAuthorizer.scala:186)
> ...
> Caused by: org.apache.zookeeper.KeeperException$BadVersionException:
> KeeperErrorCode = BadVersion for
> /kafka-acl/Topic/e6df8028-f268-408c-814e-d418e943b2fa
> at org.apache.zookeeper.KeeperException.create(KeeperException.java:115)
> at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> at org.apache.zookeeper.ZooKeeper.delete(ZooKeeper.java:873)
> at org.I0Itec.zkclient.ZkConnection.delete(ZkConnection.java:109)
> at org.I0Itec.zkclient.ZkClient$11.call(ZkClient.java:1051)
> at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:990)
> ... 18 more
> {noformat}
> I noticed this behaviour while working on another fix when running the
> {{SimpleAclAuthorizerTest}} unit tests but this can happens when running
> simultaneously the {{kafka-acls.sh}} command on different brokers in rare
> cases.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)