[
https://issues.apache.org/jira/browse/ZOOKEEPER-2778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16693231#comment-16693231
]
Michael K. Edwards edited comment on ZOOKEEPER-2778 at 11/20/18 1:35 PM:
-------------------------------------------------------------------------
May I suggest a different approach? There are three fragments of data here
(myQuorumAddr, myClientAddr, and myElectionAddr) that should be 1) updated
atomically as a group, and 2) aggressively made visible to concurrent threads
on other CPUs. There isn't really a need to lock out access to them while
other code that holds QV_LOCK runs. Seems like an ideal candidate for an
AtomicReference to an immutable POJO that holds the three addresses. Suggested
patch in https://github.com/apache/zookeeper/pull/707
was (Author: mkedwards):
May I suggest a different approach? There are three fragments of data here
(myQuorumAddr, myClientAddr, and myElectionAddr) that should be 1) updated
atomically as a group, and 2) aggressively made visible to concurrent threads
on other CPUs. There isn't really a need to lock out access to them while
other code that holds QV_LOCK runs. Seems like an ideal candidate for an
AtomicReference to an immutable POJO that holds the three addresses. Suggested
patch attached.
{{ diff --git
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java}}
{{ index 0d8a012..7bc8ea6 100644}}
{{ —
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java}}
{{ +++
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java}}
{{ @@ -42,6 +42,7 @@}}
{{ import java.util.Properties;}}
{{ import java.util.Set;}}
{{ import java.util.concurrent.atomic.AtomicInteger;}}
{{ +import java.util.concurrent.atomic.AtomicReference;}}{{import
javax.security.sasl.SaslException;}}{{@@ -121,6 +122,18 @@}}
{{ */}}
{{ private ZKDatabase zkDb;}}{{+ public static class AddressTuple {}}
{{ + public final InetSocketAddress quorumAddr;}}
{{ + public final InetSocketAddress electionAddr;}}
{{ + public final InetSocketAddress clientAddr;}}
{{ +}}
{{ + public AddressTuple(InetSocketAddress quorumAddr, InetSocketAddress
electionAddr, InetSocketAddress clientAddr)}}{{{ + this.quorumAddr =
quorumAddr; + this.electionAddr = electionAddr; + this.clientAddr = clientAddr;
+ }}}{{+ }}}
{{ +}}
{{ public static class QuorumServer {}}
{{ public InetSocketAddress addr = null;}}{{@@ -723,16 +736,14 @@ public
synchronized ServerState getPeerState(){}}{{DatagramSocket udpSocket;}}
- {{private InetSocketAddress myQuorumAddr;}}
- {{private InetSocketAddress myElectionAddr = null;}}
- {{private InetSocketAddress myClientAddr = null;}}{{ + private final
AtomicReference<AddressTuple> myAddrs = new AtomicReference<>();}}
{{/**}}
* {{Resolves hostname for a given server ID.}}{{ *}}
* {{This method resolves hostname for a given server ID in both quorumVerifer}}
* {{and lastSeenQuorumVerifier. If the server ID matches the local server ID,}}
- {{* it also updates myQuorumAddr and myElectionAddr.}}{{ + * it also updates
myAddrs.}}{{ */}}{{ public void recreateSocketAddresses(long id) {}}{{
QuorumVerifier qv = getQuorumVerifier();}}{{ @@ -741,8 +752,7 @@ public void
recreateSocketAddresses(long id) {}}{{ if (qs != null)}}{{Unknown macro: \{
qs.recreateSocketAddresses(); if (id == getId()) { - setQuorumAddress(qs.addr);
- setElectionAddress(qs.electionAddr); + setAddrs(qs.addr, qs.electionAddr,
qs.clientAddr); } }}}{{}}}
{{ @@ -756,39 +766,19 @@ public void recreateSocketAddresses(long id) {}}
{{ }}}
{{public InetSocketAddress getQuorumAddress(){}}
- {{synchronized (QV_LOCK) \{ - return myQuorumAddr; - }}}{{+ return
myAddrs.get().quorumAddr;}}
{{ }}}
- {{public void setQuorumAddress(InetSocketAddress addr){}}
- {{synchronized (QV_LOCK) \{ - myQuorumAddr = addr; - }}}
- {{}}}{{ -}}{{ public InetSocketAddress getElectionAddress(){}}
- {{synchronized (QV_LOCK) \{ - return myElectionAddr; - }}}{{+ return
myAddrs.get().electionAddr;}}
{{ }}}
- {{public void setElectionAddress(InetSocketAddress addr){}}
- {{synchronized (QV_LOCK) \{ - myElectionAddr = addr; - }}}
- {{}}}
- {{ public InetSocketAddress getClientAddress(){}}
- {{synchronized (QV_LOCK) \{ - return myClientAddr; - }}}{{+ return
myAddrs.get().clientAddr;}}
{{ }}}
- {{public void setClientAddress(InetSocketAddress addr){}}
- {{synchronized (QV_LOCK) \{ - myClientAddr = addr; - }}}{{+ public void
setAddrs(InetSocketAddress quorumAddr, InetSocketAddress electionAddr,
InetSocketAddress clientAddr)}}{{{ + myAddrs.set(new AddressTuple(quorumAddr,
electionAddr, clientAddr)); }}}
{{private int electionType;}}
{{ @@ -953,7 +943,7 @@ synchronized public void startLeaderElection()}}{{{
//}}}{{if (electionType == 0) {}}
{{ try}}{{{ - udpSocket = new DatagramSocket(myQuorumAddr.getPort()); +
udpSocket = new DatagramSocket(getQuorumAddress().getPort()); responder = new
ResponderThread(); responder.start(); }}}{{catch (SocketException e) {}}
{{ @@ -1631,9 +1621,7 @@ public QuorumVerifier setQuorumVerifier(QuorumVerifier
qv, boolean writeToDisk){}}
{{ }}}
{{ QuorumServer qs = qv.getAllMembers().get(getId());}}
{{ if (qs != null)}}{{{ - setQuorumAddress(qs.addr); -
setElectionAddress(qs.electionAddr); - setClientAddress(qs.clientAddr); +
setAddrs(qs.addr, qs.electionAddr, qs.clientAddr); }}}{{return prevQV;}}
{{ }}}
> 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
> Assignee: maoling
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 3.6.0, 3.5.5
>
> Time Spent: 1h
> 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)