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

Neha Narkhede commented on KAFKA-873:
-------------------------------------

I'm not so sure yet that moving to Curator is a good idea, at least not until 
we do a full analysis of the current zkclient problems and how Curator fixes 
those. Agreed that zkclient is not very well supported, but anytime we have 
found a serious bug, they have accepted the patch and released it. But my 
understanding is that the upside of Curator is that it includes a set of 
"recipes" for common operations that people use ZooKeeper for. 

Let me elaborate on what I think is the problem with zkclient. It wraps the 
zookeeper client APIs with the purpose of making it easy to perform common 
ZooKeeper operations. However, this limits the user to the behavior dictated by 
the wrapper, irrespective of how the underlying zookeeper library behaves. An 
example of this is the indefinite retries during a ZooKeeper disconnect. You 
may not want to retry indefinitely and might want to quit the operation after a 
timeout. Then there are various bugs introduced due to the zkclient wrapper 
design. For example, we have seen weird bugs due to the fact that zkclient 
saves the list of triggered watches in an internal queue and invokes the 
configured user callback in a background thread. 

The problems we've seen with zkclient will not be fixed with another wrapper 
(Curator). It looks like it will be better for us to just write a simple 
zookeeper client utility inside Kafka itself. If you look at zkclient, it is a 
pretty simple wrapper over the zookeeper client APIs. So this may not be a huge 
undertaking and will be a better long-term solution

> Consider replacing zkclient with curator (with zkclient-bridge)
> ---------------------------------------------------------------
>
>                 Key: KAFKA-873
>                 URL: https://issues.apache.org/jira/browse/KAFKA-873
>             Project: Kafka
>          Issue Type: Improvement
>    Affects Versions: 0.8.0
>            Reporter: Scott Clasen
>            Assignee: Grant Henke
>
> If zkclient was replaced with curator and curator-x-zkclient-bridge it would 
> be initially a drop-in replacement
> https://github.com/Netflix/curator/wiki/ZKClient-Bridge
> With the addition of a few more props to ZkConfig, and a bit of code this 
> would open up the possibility of using ACLs in zookeeper (which arent 
> supported directly by zkclient), as well as integrating with netflix 
> exhibitor for those of us using that.
> Looks like KafkaZookeeperClient needs some love anyhow...



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to