This is the amount of code a client must write to achieve this in ZooKeeper: 
https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
 
<https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java>
 - note that this class takes advantage of Curator as well. Comparing this use 
case to Kafka is mistaken. While some users might want durable and complete 
events, most really only want a simple way to follow everything that happens 
from a given parent downward. That this is effectively impossible to do in 
ZooKeeper (other than using Curator) is a large hole IMO.

-Jordan

> On Aug 13, 2019, at 8:20 AM, Jordan Zimmerman <jor...@jordanzimmerman.com> 
> wrote:
> 
> Also see https://issues.apache.org/jira/browse/ZOOKEEPER-1416 
> <https://issues.apache.org/jira/browse/ZOOKEEPER-1416>
> 
> There are many use cases where a client wants to see all events from a given 
> parent path down. The semantics of setting one-time watches on a single node 
> in ZK are cumbersome for these use cases. FWIW I had a working PR a few years 
> ago but it's fallen far behind 3.6 now.
> 
> -Jordan
> 
>> On Aug 13, 2019, at 8:18 AM, Andor Molnar <an...@apache.org 
>> <mailto:an...@apache.org>> wrote:
>> 
>> Subscriber API
>> https://issues.apache.org/jira/browse/ZOOKEEPER-153 
>> <https://issues.apache.org/jira/browse/ZOOKEEPER-153>
>> 
>> Is it supposed to be something like a generic Observer API on the client 
>> side?
>> Observers essentially consume ordered updates of ZAB, so we would need to 
>> provide a way for users to implement their own “observers”. They should be 
>> able to filter for path to be more convenient.
>> 
>> Andor
>> 
>> 
>> 
>>> On 2019. Aug 2., at 20:48, Patrick Hunt <ph...@apache.org> wrote:
>>> 
>>> Michael I think you are describing subscribe - this?
>>> https://issues.apache.org/jira/browse/ZOOKEEPER-153
>>> wasn't there some work done to keep tlogs around for a while? Or am I miss
>>> remembering? (fb folks?)
>>> 
>>> I'll also add that we haven't done any benchmarking in quite some time. It
>>> would be interesting to collect a few of these use cases from the
>>> community, esp downstreams, and evaluate performance, see if we can address.
>>> 
>>> Patrick
>>> 
>>> On Fri, Aug 2, 2019 at 11:03 AM Michael Han <h...@apache.org> wrote:
>>> 
>>>> Folks,
>>>> 
>>>> Some of you might already see this. Comments?
>>>> 
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
>>>> 
>>>> 
>>>> What caught my eyes are:
>>>> 
>>>> *Worse still, although ZooKeeper is the store of record, the state in
>>>> ZooKeeper often doesn't match the state that is held in memory in the
>>>> controller.  For example, when a partition leader changes its ISR in ZK,
>>>> the controller will typically not learn about these changes for many
>>>> seconds.  There is no generic way for the controller to follow the
>>>> ZooKeeper event log.  Although the controller can set one-shot watches, the
>>>> number of watches is limited for performance reasons.  When a watch
>>>> triggers, it doesn't tell the controller the current state-- only that the
>>>> state has changed.  By the time the controller re-reads the znode and sets
>>>> up a new watch, the state may have changed from what it was when the watch
>>>> originally fired.  If there is no watch set, the controller may not learn
>>>> about the change at all.  In some cases, restarting the controller is the
>>>> only way to resolve the discrepancy.*
>>>> 
>>>> I've seen some similar zookeeper use cases that ended up like what's
>>>> described here. How can ZooKeeper solve this? It seems to me that the only
>>>> solution is to provide linearizable read on watched operations. Thoughts?
>>>> 
>>>> Michael.
>>>> 
>> 
> 

Reply via email to