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

Jessica Cheng Mallet commented on SOLR-7844:
--------------------------------------------

[[email protected]], not sure if this is intended--looks like the newly 
added ShardLeaderElectionContextBase.cancelElection now blindly deletes the 
leader node, which sounds just as dangerous. From your comment it seems like 
you just wanted it to expire out, so I'm wondering if it's just a merge bug or 
something.

In general, I think it'd make a lot of sense to predicate the writing of the 
leader node on the election node still having the same session as the thread 
thinks (using the same zookeeper multi-transactional semantics as in 
ZkController.markShardAsDownIfLeader), so that a thread that went GCing before 
writing the leader node will fail when it comes back since its election node 
will have expired. 

> Zookeeper session expiry during shard leader election can cause multiple 
> leaders.
> ---------------------------------------------------------------------------------
>
>                 Key: SOLR-7844
>                 URL: https://issues.apache.org/jira/browse/SOLR-7844
>             Project: Solr
>          Issue Type: Bug
>    Affects Versions: 4.10.4
>            Reporter: Mike Roberts
>            Assignee: Mark Miller
>             Fix For: Trunk, 5.4
>
>         Attachments: SOLR-7844.patch, SOLR-7844.patch
>
>
> If the ZooKeeper session expires for a host during shard leader election, the 
> ephemeral leader_elect nodes are removed. However the threads that were 
> processing the election are still present (and could believe the host won the 
> election). They will then incorrectly create leader nodes once a new 
> ZooKeeper session is established.
> This introduces a subtle race condition that could cause two hosts to become 
> leader.
> Scenario:
> a three machine cluster, all of the machines are restarting at approximately 
> the same time.
> The first machine starts, writes a leader_elect ephemeral node, it's the only 
> candidate in the election so it wins and starts the leadership process. As it 
> knows it has peers, it begins to block waiting for the peers to arrive.
> During this period of blocking[1] the ZK connection drops and the session 
> expires.
> A new ZK session is established, and ElectionContext.cancelElection is 
> called. Then register() is called and a new set of leader_elect ephemeral 
> nodes are created.
> During the period between the ZK session expiring, and new set of 
> leader_elect nodes being created the second machine starts.
> It creates its leader_elect ephemeral nodes, as there are no other nodes it 
> wins the election and starts the leadership process. As its still missing one 
> of its peers, it begins to block waiting for the third machine to join.
> There is now a race between machine1 & machine2, both of whom think they are 
> the leader.
> So far, this isn't too bad, because the machine that loses the race will fail 
> when it tries to create the /collection/name/leader/shard1 node (as it 
> already exists), and will rejoin the election.
> While this is happening, machine3 has started and has queued for leadership 
> behind machine2.
> If the loser of the race is machine2, when it rejoins the election it cancels 
> the current context, deleting it's leader_elect ephemeral nodes.
> At this point, machine3 believes it has become leader (the watcher it has on 
> the leader_elect node fires), and it runs the LeaderElector::checkIfIAmLeader 
> method. This method DELETES the current /collection/name/leader/shard1 node, 
> then starts the leadership process (as all three machines are now running, it 
> does not block to wait).
> So, machine1 won the race with machine2 and declared its leadership and 
> created the nodes. However, machine3 has just deleted them, and recreated 
> them for itself. So machine1 and machine3 both believe they are the leader.
> I am thinking that the fix should be to cancel & close all election contexts 
> immediately on reconnect (we do cancel them, however it's run serially which 
> has blocking issues, and just canceling does not cause the wait loop to 
> exit). That election context logic already has checks on the closed flag, so 
> they should exit if they see it has been closed.
> I'm working on a patch for this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to