[
https://issues.apache.org/jira/browse/ZOOKEEPER-2080?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15482766#comment-15482766
]
Michael Han commented on ZOOKEEPER-2080:
----------------------------------------
bq. The shut down of leader election doesn't need to be in the synchronized
block.
This is what the patch did, by moving the shut down of LE out of the sync
block.
bq. it should be volatile.
Yes, agreed. Will update patch.
bq. is it possible that due to a race we miss that we are supposed to start LE?
Possible in theory - an example:
* shuttingDownLE is set to true in
[FastLeaderElection|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L302]
* There are a couple of other places in Leader / Follower where
[processReconfig|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L302]
will be called with restart LE flag set to true, which will set the
shuttingDownLE flag to false.
* So in theory, between we set shuttingDownLE to true in
[FastLeaderElection|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L302]
and we ran into check the same flag in
[QuorumPeer|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1066],
there could be concurrent operations which involves processReconfig that
change the state of shuttingDownLE from true to false. As a result, we miss the
event of the state change of shuttingDownLE in QuorumPeer's main loop.
Now does it matter if we miss such event due to interleaving operations from
processReconfig set shuttingDownLE from true to false? I am not sure, as I am
still learning the semantic of reconfig and FLE to make a better guess here.
Other notes:
* Do we need to synchronize on this code in QuorumPeer? I think probably we
should, because similar 'test and set' operation around shuttingDownLE in
QuorumPeer (such as
[restartLeaderElection|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1419]
is synchronized, which makes sense (as typically in multithreading, what we
need to protected is not just the signal variable but also the executions that
depends on the value of the variable).
* I think we have another potential dead lock in
[QuorumPeer.restartLeaderElection|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1415].
Here the method is synchronized, but part of its execution requires invokes of
[FastLeaderElection.shutdown|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1418],
which, as previously analyzed could end up at a place in
[QuorumCnxManager|https://github.com/apache/zookeeper/blob/3c37184e83a3e68b73544cebccf9388eea26f523/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java#L475]
where it requires obtain same QuorumPeer lock. I don't think we are shot by
this yet, could either because we don't have test case cover such case, or
could because that is just a theory that in practice we will never end up with
such code execution flow.
* While looking through code, I find it is hard to reason about current
behavior due to the mixed state changes that could be done by many parties. I
am thinking if there is better ways of doing this, for example using an actor
based mode to restructure the code such that we have an event driven
architecture where each object has a main thread which only dispatch events and
entities (such as QuorumPeer and FastLeaderElection) will communicate each
other by passing event messages - there would not be any shared state and state
changes for each entities will then be easier to reason about. I think I will
have a better evaluation after more investigation around FLE codebase.
> ReconfigRecoveryTest fails intermittently
> -----------------------------------------
>
> Key: ZOOKEEPER-2080
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2080
> Project: ZooKeeper
> Issue Type: Sub-task
> Reporter: Ted Yu
> Assignee: Michael Han
> Fix For: 3.5.3, 3.6.0
>
> Attachments: ZOOKEEPER-2080.patch, ZOOKEEPER-2080.patch,
> ZOOKEEPER-2080.patch, ZOOKEEPER-2080.patch,
> jacoco-ZOOKEEPER-2080.unzip-grows-to-70MB.7z, repro-20150816.log,
> threaddump.log
>
>
> I got the following test failure on MacBook with trunk code:
> {code}
> Testcase: testCurrentObserverIsParticipantInNewConfig took 93.628 sec
> FAILED
> waiting for server 2 being up
> junit.framework.AssertionFailedError: waiting for server 2 being up
> at
> org.apache.zookeeper.server.quorum.ReconfigRecoveryTest.testCurrentObserverIsParticipantInNewConfig(ReconfigRecoveryTest.java:529)
> at
> org.apache.zookeeper.JUnit4ZKTestRunner$LoggedInvokeMethod.evaluate(JUnit4ZKTestRunner.java:52)
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)