Skip to content

Conversation

@razinbouzar
Copy link
Contributor

Fixes #16411 .

Description

Introduces a new private method handleConnectionStateChanged to handle connection state changes, which recreates the leader latch if the connection state is SUSPENDED or LOST.

Fixed the bug ...

Renamed the class ...

Added a forbidden-apis entry ...

Release note


Key changed/added classes in this PR
  • MyFoo
  • OurBar
  • TheirBaz

This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • a release note entry in the PR description.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

Razin Bouzar added 2 commits May 31, 2024 12:34
Added listener method that tracks ZK leader state
@razinbouzar
Copy link
Contributor Author

@kfaraz I took an attempt at resolving this if you can take a look and provide feedback it would be appreciated! Thank you!

@kfaraz kfaraz self-requested a review May 31, 2024 17:20
@kfaraz
Copy link
Contributor

kfaraz commented May 31, 2024

Thanks a lot, @razinbouzar ! You beat me to it. 🙂

I have been doing some investigation on this. Overall, your solution makes sense to me. But I am still going to try it out in my local setup that I have been using for testing.

One important thing that I have noticed is that the isLeader() method may be called on the leader latch listener even after the latch has been closed. So we would need to make sure that if this method is called on a closed latch, we just ignore that event.

I will share more details here in a bit.

Copy link
Contributor

@kfaraz kfaraz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have left some suggestions. Overall approach makes sense to me.

I still need to test out the changes in my local setup.
An alternative solution I was thinking of was just to recreate the latch in notLeader() method. We would need to evaluate if one is better than the other.

@kfaraz kfaraz requested a review from gianm May 31, 2024 17:35
@kfaraz
Copy link
Contributor

kfaraz commented May 31, 2024

One important thing that I have noticed is that the isLeader() method may be called on the leader latch listener even after the latch has been closed. So we would need to make sure that if this method is called on a closed latch, we just ignore that event.

For more information, I encountered this in my local testing when I was trying to recreate the latch in notLeader().

Timeline:

  1. [curator-thread] Leader loses connection
  2. Leader gets notified with notLeader()
  3. [curator-thread] Connection is re-established
  4. Create new latch, close old one. This immediately causes some other node to become leader.
  5. Start new latch after a delay to allow other nodes to become leader.
  6. isLeader() is called on old latch which is already closed. This leads to double leaders if not properly handled. This seems like another curator bug since there is no point in calling isLeader() if there is already another leader. (step 4 above).

An example sequence of events that I observed

2024-05-30T14:19:11,556 WARN [main-SendThread(localhost:2181)] org.apache.zookeeper.ClientCnxn - Session 0x1002399b0fc0000 for server localhost/127.0.0.1:2181, Closing socket connection. Attempting reconnect except it is a SessionExpiredException.
2024-05-30T14:19:11,670 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - [http://localhost:8081][1] Giving up leadership
2024-05-30T14:19:11,754 INFO [main-SendThread(localhost:2183)] org.apache.zookeeper.ClientCnxn - Session establishment complete on server localhost/0:0:0:0:0:0:0:1:2183, session id = 0x1002399b0fc0000, negotiated timeout = 30000
2024-05-30T14:19:11,766 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - [http://localhost:8081][1] Recreating leader latch to allow other nodes to become leader.
2024-05-30T14:19:13,379 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - [http://localhost:8081][1] Now starting the latch
2024-05-30T14:19:13,379 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - [http://localhost:8081][1] I am now the leader. Latch state[CLOSED]

Just for clarification, the scenario described above is slightly different from the one we are trying to solve.
In the original problem, the connection is not re-established until the znodes created by the current leader are just about to expire and it tries to reacquire leadership.

Razin Bouzar and others added 2 commits May 31, 2024 17:56
razinbouzar and others added 2 commits June 2, 2024 01:07
- Cleanup file formatting and comments
- Reduce complexity of the first go by calling the recreateLeaderLatch in the notLeader() method
…rDruidLeaderSelector.java

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
Copy link
Contributor

@kfaraz kfaraz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add a unit test that verifies the new behaviour.
Also the case state = CLOSED still needs to be handled in the isLeader() method of the listener.

- Remove handleConnectionStateChagned method

- Remove duplicate code and use recreate leader latch method

- Handle LeaderLatch.State.CLOSED in the isLeader() function, log a warning.
Remove unused import
Copy link
Contributor

@kfaraz kfaraz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have done some basic testing changes with these changes and they look good.
I have a follow up PR #16544 to this where I intend to add tests to verify the behaviour and fix other race conditions.

Thanks a lot for the changes, @razinbouzar !

@kfaraz kfaraz merged commit 844b217 into apache:master Jun 7, 2024
@kfaraz kfaraz mentioned this pull request Jun 18, 2024
10 tasks
@kfaraz kfaraz added this to the 31.0.0 milestone Oct 4, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 Coordinators Elected Leader

2 participants