zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jordan Zimmerman <jor...@jordanzimmerman.com>
Subject Re: [jira] [Comment Edited] (ZOOKEEPER-2169) Enable creation of nodes with TTLs
Date Mon, 01 Aug 2016 20:54:50 GMT
But that’s a common pattern with ZooKeeper. Curator has PersistentEphemeralNode for just
this type of use-case. 

> On Aug 1, 2016, at 3:41 PM, Stevo Slavić <sslavic@gmail.com> wrote:
> 
> Imagine a stateless app in front of Kafka/ZooKeeper cluster. It handles
> request to create a temporary topic. It creates temporary topic, which
> results in TTL node in ZK, signals success to the caller. And then app
> crashes or gets restarted or gets relocated to run on different host.
> Starting it again, with no state, will have no idea about TTL node it
> created before, TTL node might have died in the meantime, so no
> on-TTL-node-deletion trigger gets handled and as result ZK has junk content
> wasting resources.
> 
> On Mon, Aug 1, 2016 at 9:58 PM, Jordan Zimmerman <jordan@jordanzimmerman.com
>> wrote:
> 
>> I think it would be simpler and less disruptive to use the existing
>> Watcher mechanism. Treat it as a deleted node watcher. When the
>> TTL/container node gets deleted the watcher triggers. Actually, come to
>> think of it, wouldn’t that work already? Like any other node, after
>> creating it you can call exists() to set a watcher. Do we really need new
>> behavior here?
>> 
>> -Jordan
>> 
>>> On Aug 1, 2016, at 11:02 AM, Stevo Slavić <sslavic@gmail.com> wrote:
>>> 
>>> Yes, all server side. Client that successfully creates TTL node and
>>> registers such "trigger", could disconnect - trigger should fire on TTL
>>> node deletion and be handled on server side only. So server watching and
>>> handling event. It shouldn't happen that TTL node gets deleted and event
>>> does not get handled - both deletion of TTL node and handling of a
>> trigger
>>> both should either succeed or fail.
>>> 
>>> On Mon, Aug 1, 2016 at 5:37 PM, Jordan Zimmerman <
>> jordan@jordanzimmerman.com
>>>> wrote:
>>> 
>>>> That’s an interesting idea - so a watcher for container expirations?
>>>> 
>>>> -Jordan
>>>> 
>>>>> On Aug 1, 2016, at 2:20 AM, Stevo Slavić <sslavic@gmail.com> wrote:
>>>>> 
>>>>> Hello Apache ZooKeeper developers,
>>>>> 
>>>>> Thinking, for a use case like support temporary topics in Apache Kafka,
>>>>> which could be based on ZooKeeper TTL feature, might be useful to be
>> able
>>>>> to register a ZooKeeper "trigger" once TTL expires for a node - e.g.
in
>>>>> same transaction that deletes temporary node, create another persistent
>>>>> node (request to delete the topic). Of course one could workaround
>> this,
>>>> by
>>>>> creating persistent and TTL node, and check if there is persistent node
>>>>> without matching temporary node, but option with trigger would have
>> been
>>>>> better/easier from consistency point of view.
>>>>> 
>>>>> What do you think about the idea?
>>>>> 
>>>>> Kind regards,
>>>>> Stevo Slavic.
>>>>> 
>>>>> 
>>>>> On Mon, Aug 1, 2016 at 3:31 AM, Raul Gutierrez Segales (JIRA) <
>>>>> jira@apache.org> wrote:
>>>>> 
>>>>>> 
>>>>>>  [
>>>>>> 
>>>> 
>> https://issues.apache.org/jira/browse/ZOOKEEPER-2169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15401429#comment-15401429
>>>>>> ]
>>>>>> 
>>>>>> Raul Gutierrez Segales edited comment on ZOOKEEPER-2169 at 8/1/16
1:31
>>>> AM:
>>>>>> 
>>>> 
>> ---------------------------------------------------------------------------
>>>>>> 
>>>>>> [~fpj]: it's here: https://reviews.apache.org/r/46983/.
>>>>>> 
>>>>>> cc: [~randgalt]
>>>>>> 
>>>>>> 
>>>>>> was (Author: rgs):
>>>>>> [~fpj]: it's here.
>>>>>> 
>>>>>> cc: [~randgalt]
>>>>>> 
>>>>>>> Enable creation of nodes with TTLs
>>>>>>> ----------------------------------
>>>>>>> 
>>>>>>>              Key: ZOOKEEPER-2169
>>>>>>>              URL:
>>>>>> https://issues.apache.org/jira/browse/ZOOKEEPER-2169
>>>>>>>          Project: ZooKeeper
>>>>>>>       Issue Type: New Feature
>>>>>>>       Components: c client, java client, jute, server
>>>>>>> Affects Versions: 3.6.0
>>>>>>>         Reporter: Camille Fournier
>>>>>>>         Assignee: Jordan Zimmerman
>>>>>>>          Fix For: 3.6.0
>>>>>>> 
>>>>>>>      Attachments: ZOOKEEPER-2169-2.patch, ZOOKEEPER-2169-3.patch,
>>>>>> ZOOKEEPER-2169-4.patch, ZOOKEEPER-2169-5.patch, ZOOKEEPER-2169.patch
>>>>>>> 
>>>>>>> 
>>>>>>> As a user, I would like to be able to create a node that is NOT
tied
>> to
>>>>>> a session but that WILL expire automatically if action is not taken
by
>>>> some
>>>>>> client within a time window.
>>>>>>> I propose this to enable clients interacting with ZK via http
or
>> other
>>>>>> "thin clients" to create ephemeral-like nodes.
>>>>>>> Some ideas for the design, up for discussion:
>>>>>>> The node should support all normal ZK node operations including
ACLs,
>>>>>> sequential key generation, etc, however, it should not support the
>>>>>> ephemeral flag. The node will be created with a TTL that is updated
>> via
>>>> a
>>>>>> refresh operation.
>>>>>>> The ZK quorum will watch this node similarly to the way that
it
>> watches
>>>>>> for session liveness; if the node is not refreshed within the TTL,
it
>>>> will
>>>>>> expire.
>>>>>>> QUESTIONS:
>>>>>>> 1) Should we let the refresh operation set the TTL to a different
>> base
>>>>>> value?
>>>>>>> 2) If so, should the setting of the TTL to a new base value cause
a
>>>>>> watch to fire?
>>>>>>> 3) Do we want to allow these nodes to have children or prevent
this
>>>>>> similar to ephemeral nodes?
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> This message was sent by Atlassian JIRA
>>>>>> (v6.3.4#6332)
>>>>>> 
>>>> 
>>>> 
>> 
>> 


Mime
View raw message