[
https://issues.apache.org/jira/browse/CURATOR-495?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jordan Zimmerman updated CURATOR-495:
-------------------------------------
Description:
In trying to figure out why {{TestInterProcessSemaphoreMutex}} is so flakey
I've come across a fairly serious edge case in how several of our recipes work.
You can see the issue in {{InterProcessSemaphoreV2}} (which is what
{{InterProcessSemaphoreMutex}} uses internally). Look here:
[InterProcessSemaphoreV2.java|https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java#L373]
The code synchronizes and then does {{client.getChildren()...}}. This is where
the problem is. If there are connection problems inside of getChildren() the
retry policy will do configured sleeping, retries, etc. Importantly, this is
all done while the thread doing the retries holds InterProcessSemaphoreV2's
monitor. If the ZK connection is repaired past the session timeout, ZK will
eventually call InterProcessSemaphoreV2's watcher with an Expired message.
InterProcessSemaphoreV2's watcher calls this method:
{code}
private synchronized void notifyFromWatcher()
{
notifyAll();
}
{code}
You can see that this is a race. The thread doing "getChildren" is holding the
monitor and is in a retry loop waiting for the connection to be repaired.
However, ZK's event loop is trying to obtain that same monitor as a result of
trying to call the synchronized notifyFromWatcher(). This means that the retry
policy will always fail because ZK's event loop is tied up until that thread
exists. Worse still, if someone were to use a retry policy of "RetryForever"
they'd have a deadlock.
This pattern is in about 10 files or so. I'm trying to think of a workaround.
One possibility is to use a separate thread for this type of notification. i.e.
notifyFromWatcher() would just signal another thread that the notifyAll() needs
to be called. This would unblock ZK's event thread so that things can progress.
I'll play around with this.
was:
In trying to figure out why {{TestInterProcessSemaphoreMutex}} is so flakey
I've come across a fairly serious edge case in how several of our recipes work.
You can see the issue in {{InterProcessSemaphoreV2}} (which is what
{{InterProcessSemaphoreMutex}} uses internally). Look here:
[InterProcessSemaphoreV2.java|https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java#L373]
The code synchronizes and then does {{client.getChildren()...}}. This is where
the problem is. If there are connection problems inside of getChildren() the
retry policy will do configured sleeping, retries, etc. Importantly, this is
all done while the thread doing the retires holds InterProcessSemaphoreV2's
monitor. If the ZK connection is repaired past the session timeout, ZK will
eventually call InterProcessSemaphoreV2's watcher with an Expired message.
InterProcessSemaphoreV2's watcher calls this method:
{code}
private synchronized void notifyFromWatcher()
{
notifyAll();
}
{code}
You can see that this is a race. The thread doing "getChildren" is holding the
monitor and is in a retry loop waiting for the connection to be repaired.
However, ZK's event loop is trying to obtain that same monitor as a result of
trying to call the synchronized notifyFromWatcher(). This means that the retry
policy will always fail because ZK's event loop is tied up until that thread
exists. Worse still, if someone were to use a retry policy of "RetryForever"
they'd have a deadlock.
This pattern is in about 10 files or so. I'm trying to think of a workaround.
One possibility is to use a separate thread for this type of notification. i.e.
notifyFromWatcher() would just signal another thread that the notifyAll() needs
to be called. This would unblock ZK's event thread so that things can progress.
I'll play around with this.
> Race and possible dead locks with RetryPolicies and several Curator Recipes
> ---------------------------------------------------------------------------
>
> Key: CURATOR-495
> URL: https://issues.apache.org/jira/browse/CURATOR-495
> Project: Apache Curator
> Issue Type: Bug
> Components: Recipes
> Affects Versions: 4.0.1
> Reporter: Jordan Zimmerman
> Assignee: Jordan Zimmerman
> Priority: Blocker
> Fix For: 4.1.0
>
>
> In trying to figure out why {{TestInterProcessSemaphoreMutex}} is so flakey
> I've come across a fairly serious edge case in how several of our recipes
> work. You can see the issue in {{InterProcessSemaphoreV2}} (which is what
> {{InterProcessSemaphoreMutex}} uses internally). Look here:
> [InterProcessSemaphoreV2.java|https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java#L373]
> The code synchronizes and then does {{client.getChildren()...}}. This is
> where the problem is. If there are connection problems inside of
> getChildren() the retry policy will do configured sleeping, retries, etc.
> Importantly, this is all done while the thread doing the retries holds
> InterProcessSemaphoreV2's monitor. If the ZK connection is repaired past the
> session timeout, ZK will eventually call InterProcessSemaphoreV2's watcher
> with an Expired message. InterProcessSemaphoreV2's watcher calls this method:
> {code}
> private synchronized void notifyFromWatcher()
> {
> notifyAll();
> }
> {code}
> You can see that this is a race. The thread doing "getChildren" is holding
> the monitor and is in a retry loop waiting for the connection to be repaired.
> However, ZK's event loop is trying to obtain that same monitor as a result of
> trying to call the synchronized notifyFromWatcher(). This means that the
> retry policy will always fail because ZK's event loop is tied up until that
> thread exists. Worse still, if someone were to use a retry policy of
> "RetryForever" they'd have a deadlock.
> This pattern is in about 10 files or so. I'm trying to think of a workaround.
> One possibility is to use a separate thread for this type of notification.
> i.e. notifyFromWatcher() would just signal another thread that the
> notifyAll() needs to be called. This would unblock ZK's event thread so that
> things can progress. I'll play around with this.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)