[
https://issues.apache.org/jira/browse/CURATOR-476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Rohan Suri updated CURATOR-476:
-------------------------------
Description:
h5. What's happening:
When PathChildrenCache is started with
PathChildrenCache.StartMode.POST_INITIALIZED_EVENT there could be the following
scenario:
1. PCC calls getChildren("../parent") and gets the list of it's children
2. PCC calls processChildren(...) where it puts them into it's \{initialSet} –
set of children whose data it will fetch and cache
3. Later in the same method it calls getDataAndStat(..."/parent/somechild")
where it calls getData(...) on this child and registers a dataWatcher for this
child
Midst of step 2&3, child node could get deleted – but since step 3 hasn't
completed yet, the dataWatcher isn't triggered (since it is not even
registered) and we miss the NodeDeleted event.
Since our \{initialSet} still contains the child path, the initialized event is
never fired.
h5. The fix:
As of now there's only one codepath to removing a path from the \{initialSet}
(done from the data watcher)
The remove should also be called in the callback of our
getData(..."/parent/somechild") being done in Step 3 if the
resultCode=-101(KeeperException.NoNode)
was:
PathChildrenCache started with
PathChildrenCache.StartMode.POST_INITIALIZED_EVENT there could be the following
scenario:
1. PCC calls getChildren("../parent") and gets the list of it's children
2. PCC calls processChildren(...) where it puts them into it's \{initialSet} –
set of children whose data it will fetch and cache
3. Later in the same method it calls getDataAndStat(..."/parent/somechild")
where it calls getData(...) on this child and registers a dataWatcher for this
child
Midst of step 2&3, child node could get deleted – but since step 3 hasn't
completed yet, the dataWatcher isn't triggered (since it is not even registered)
and we miss the NodeDeleted event.
Since our \{initialSet} still contains the child path, the initialized event is
never fired.
The fix:
As of now there's only one codepath to removing a path from the \{initialSet}
(from the data watcher)
The remove should also be called in the callback of our
getData(..."/parent/somechild") being done in Step 3 if the
resultCode=-101(KeeperException.NoNode)
> PathChildrenCache never get initialized if a child dies before a data watch
> is registered for it
> ------------------------------------------------------------------------------------------------
>
> Key: CURATOR-476
> URL: https://issues.apache.org/jira/browse/CURATOR-476
> Project: Apache Curator
> Issue Type: Bug
> Components: Recipes
> Reporter: Rohan Suri
> Priority: Major
>
> h5. What's happening:
> When PathChildrenCache is started with
> PathChildrenCache.StartMode.POST_INITIALIZED_EVENT there could be the
> following scenario:
> 1. PCC calls getChildren("../parent") and gets the list of it's children
> 2. PCC calls processChildren(...) where it puts them into it's \{initialSet}
> – set of children whose data it will fetch and cache
> 3. Later in the same method it calls getDataAndStat(..."/parent/somechild")
> where it calls getData(...) on this child and registers a dataWatcher for
> this child
> Midst of step 2&3, child node could get deleted – but since step 3 hasn't
> completed yet, the dataWatcher isn't triggered (since it is not even
> registered) and we miss the NodeDeleted event.
> Since our \{initialSet} still contains the child path, the initialized event
> is never fired.
> h5. The fix:
> As of now there's only one codepath to removing a path from the \{initialSet}
> (done from the data watcher)
> The remove should also be called in the callback of our
> getData(..."/parent/somechild") being done in Step 3 if the
> resultCode=-101(KeeperException.NoNode)
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)