zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Michael K. Edwards (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (ZOOKEEPER-2778) Potential server deadlock between follower sync with leader and follower receiving external connection requests.
Date Tue, 20 Nov 2018 23:45:00 GMT

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

Michael K. Edwards edited comment on ZOOKEEPER-2778 at 11/20/18 11:44 PM:
--------------------------------------------------------------------------

>From what I'm seeing, it would be a crashing bug for {{getQuorumAddress()}} (which cannot
be marked {{protected}}, because it's called by has-a holders of a {{QuorumPeer}} reference
rather than by is-a subclasses of {{QuorumPeer}}, but can and should be package-private) to
be called before the addresses are set.  The only call to {{getClientAddress()}} (which should
be {{private}}) is in {{processReconfig()}}, and it's appropriate for it to return {{null}}
if called early.  This leaves {{getElectionAddress()}}, which again is pseudo-protected and
would produce a crash if called before the addresses are set.

So the actual problem here is that, if the election address is not yet known, there's no safe
return value from {{getElectionAddress()}} in the race scenario cited in the bug description.
 This "fix" – hanm's or mine – will turn it into an NPE instead of a deadlock.

This might be addressable by ensuring that code that needs the {{QV_LOCK}} for a {{QuorumPeer}}
associated with a {{QuorumCnxManager}} (to protect the macroscopic critical sections in {{QuorumCnxManager.connectOne()}},
{{QuorumPeer.setLastSeenQuorumVerifier()}}, and {{QuorumPeer.setQuorumVerifier()}}) always
takes the lock on the {{QuorumCnxManager}} instance first.  Looking into that.


was (Author: mkedwards):
>From what I'm seeing, it would be a crashing bug for `getQuorumAddress()` (which cannot
be marked `protected`, because it's called by has-a holders of a QuorumPeer reference rather
than by is-a subclasses of QuorumPeer, but can and should be package-private) to be called
before the addresses are set.  The only call to `getClientAddress()` (which should be `private`)
is in `processReconfig()`, and it's appropriate for it to return `null` if called early. 
This leaves `getElectionAddress()`, which again is pseudo-protected and would produce a crash
if called before the addresses are set.

So the actual problem here is that, if the election address is not yet known, there's no safe
return value from `getElectionAddress()` in the race scenario cited in the bug description.
 This "fix" – hanm's or mine – will turn it into an NPE instead of a deadlock.

This might be addressable by ensuring that code that needs the `QV_LOCK` for a `QuorumPeer`
associated with a `QuorumCnxManager` (to protect the macroscopic critical sections in `QuorumCnxManager.connectOne()`,
`QuorumPeer.setLastSeenQuorumVerifier()`, and `QuorumPeer.setQuorumVerifier()`) always takes
the lock on the `QuorumCnxManager` instance first.  Looking into that.

> Potential server deadlock between follower sync with leader and follower receiving external
connection requests.
> ----------------------------------------------------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-2778
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2778
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: quorum
>    Affects Versions: 3.5.3
>            Reporter: Michael Han
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 3.6.0, 3.5.5
>
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> It's possible to have a deadlock during recovery phase. 
> Found this issue by analyzing thread dumps of "flaky" ReconfigRecoveryTest [1]. . Here
is a sample thread dump that illustrates the state of the execution:
> {noformat}
>     [junit]  java.lang.Thread.State: BLOCKED
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumPeer.getElectionAddress(QuorumPeer.java:686)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumCnxManager.initiateConnection(QuorumCnxManager.java:265)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumCnxManager.connectOne(QuorumCnxManager.java:445)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumCnxManager.receiveConnection(QuorumCnxManager.java:369)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumCnxManager$Listener.run(QuorumCnxManager.java:642)
>     [junit] 
>     [junit]  java.lang.Thread.State: BLOCKED
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumCnxManager.connectOne(QuorumCnxManager.java:472)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumPeer.connectNewPeers(QuorumPeer.java:1438)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumPeer.setLastSeenQuorumVerifier(QuorumPeer.java:1471)
>     [junit]         at  org.apache.zookeeper.server.quorum.Learner.syncWithLeader(Learner.java:520)
>     [junit]         at  org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:88)
>     [junit]         at  org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1133)
> {noformat}
> The dead lock happens between the quorum peer thread which running the follower that
doing sync with leader work, and the listener of the qcm of the same quorum peer that doing
the receiving connection work. Basically to finish sync with leader, the follower needs to
synchronize on both QV_LOCK and the qmc object it owns; while in the receiver thread to finish
setup an incoming connection the thread needs to synchronize on both the qcm object the quorum
peer owns, and the same QV_LOCK. It's easy to see the problem here is the order of acquiring
two locks are different, thus depends on timing / actual execution order, two threads might
end up acquiring one lock while holding another.
> [1] org.apache.zookeeper.server.quorum.ReconfigRecoveryTest.testCurrentServersAreObserversInNextConfig



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message