Skip to content

DruidLeaderSelector interface for leader election and Curator based impl.#4699

Merged
drcrallen merged 9 commits intoapache:masterfrom
himanshug:membership_pr2_1
Sep 1, 2017
Merged

DruidLeaderSelector interface for leader election and Curator based impl.#4699
drcrallen merged 9 commits intoapache:masterfrom
himanshug:membership_pr2_1

Conversation

@himanshug
Copy link
Copy Markdown
Contributor

@himanshug himanshug commented Aug 17, 2017

Follow up to #4634

Introduces DruidLeaderSelector interface and curator based implementation used at coordinator (in DruidCoordinator) and overlord (in TaskMaster).

note for 0.11.0 release upgrade:
Because overlord leader election algorithm changes with this patch, so it is required to shutdown all overlords and upgrade them and start. There should be no time when two different overlords are not running 0.11.0 during the upgrade.
Note that at least one overlord should be brought up as quickly as possible after shutting them all down so that peons, tranquility etc continue to work after some retries.
druid.zk.paths.indexer.leaderLatchPath is ignored now.

@cheddar
Copy link
Copy Markdown
Contributor

cheddar commented Aug 23, 2017

👍

}

leader = true;
term++;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Doesn't zookeeper already have a term concept? is there any benefit to using another term instead of inheriting the one in zookeeper?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

curator LeaderLatch does not expose any term concept. You are probably talking about zookeeper quorums internal leader election.
also, this term is created to keep behavior same as in existing code in DruidCoordinator which was maintaining a local term to keep things in check.

leader = false;
try {
//Small delay before starting the latch so that others waiting are chosen to become leader.
Thread.sleep(1000);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

An unlucky GC could easily induce a pause of 1 second. Is there a way to do this without the mandatory pause?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this is probably only place in the whole PR where I have intentionally introduced something that did not exist before. and, this is done to fix following scenario.

  • say this coordinator node has a problem in becoming leader (e.g. not being able to reach database or something like that) then this needs to tell curator to give up its leadership so that someone else becomes the leader... however if it is too quick in starting next latch then curator may end up choosing this node leader again and keep on repeating the cycle.

this simple artificial pause solves the problem without introducing any issues.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I can't think of a better way right now without introducing a lot of potentially unhelpful complexity.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

#3428 is solved by this?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

overlord was using curator LeaderSelector and not LeaderLatch , which is a completely different implementation. Most likely #3428 would get solved by this.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

For stuff like this, a random sleep works better than a fixed sleep, since a fixed sleep can cause the same sequence of leader changes to play out over and over again. And I think a random sleep is a totally valid mechanism of trying to jolt a leader-election system onto a better leader. It's pretty common in leader-election algorithms.

In this case, maybe a random sleep between 500ms and 5000ms would work.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

makes sense, changed random sleep between 1000ms to 5000ms .
min is 1000ms and not 500ms just to be safer and also because I tested with that value mostly.

Copy link
Copy Markdown
Contributor

@drcrallen drcrallen left a comment

Choose a reason for hiding this comment

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

@himanshug can you please fill out the interface method docs for the new interface? it is impossible to review the PR without actually knowing what the interface is supposed to be doing.

}
}
},
Execs.singleThreaded(StringUtils.format("LeaderSelector[%s]", latchPath))
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This is not lifecycled at all. is it possible to have the lifecycle here controlled somehow, to make sure that the executor is not running when this jvm is not the leader?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Also, is it possible to add the term to the string format?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this executor does not change at each term , so term is not a part of the name . also it consists of daemon thread and not needed to be explicitly stopped but only when jvm is shutting down.
note that it needs to be running even when this node is not the leader to get the notification that this node should now become leader , it is the "watcher" passed to curator LeaderLatch .

Copy link
Copy Markdown
Contributor

@drcrallen drcrallen Aug 29, 2017

Choose a reason for hiding this comment

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

Gotcha, I misunderstood the scope of this object

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Copy link
Copy Markdown
Contributor Author

@himanshug himanshug Aug 30, 2017

Choose a reason for hiding this comment

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

hmmm, given that this code existed before and has been working fine, earlier I thought that these would become garbage and GC'd away as no reference to these executors is being maintained.
however I did some tests today to verify that and it appears that yes these objects don't get GC'd and all of them linger in the jvn, don't cause anything bad but waste some memory. I will update the code to fix this. thanks.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@drcrallen updated code to not recreate executor every time but just once.

}

@Override
public String getCurrentLeader()
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@Nullable

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

sure, will add

@Override
public boolean isLeader()
{
return leader;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

if the leader status is being contested, should this block until the contest is settled?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

while under contest at curator/zookeeper level, this would be false and that is in line with current semantics .


/**
*/
public interface DruidLeaderSelector
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This class needs a LOT more documentation on what the method contracts are

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

sure, will add

}
}

private static class DruidLeaderSelectorProvider implements Provider<DruidLeaderSelector>
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

usually to ease testing there is a constructor annotated with @Inject that populates private final fields

Copy link
Copy Markdown
Contributor Author

@himanshug himanshug Aug 24, 2017

Choose a reason for hiding this comment

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

this is a utility class only and constructor is explicitly called to provide latchPath which are not part of guice binding , other things are injected and that is why it looks a bit different.

return ScheduledExecutors.Signal.STOP;
}
@Override
public ScheduledExecutors.Signal call()
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

what is the intended behavior if leadership is lost during this call?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this scheduled callable would stop repeating itself ... and will get rescheduled when this node becomes leader again. i think current semantics are retained.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, this also means if lookup management is taking a long time (minutes) then this is going to continue doing lookup management until it either completes or errors out, then it will give up leadership.

In such a scenario it is possible for the new leader to also do lookup management at the same time.

Is there a way to mitigate such a scenario?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

well it is mitigated in the sense that even if multiple coordinators are doing lookup management , nothing bad happens... in the worst case, downstream nodes would receive request to load same lookup multiple times and those requests would be ignored.


public String getOverlordPath()
{
return defaultPath("overlord");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

this is different than the other patterns which have a settable path, why does this one need to require only the default path?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

i intentionally did not make the latch path configurable . IMO only base zookeeper path should be configurable but all the other locations inside that base path should be dictated by Druid. I am not sure if there is any value in making internal paths configurable.
My understanding is that existing paths were made configurable only to support migration and not that there is any use case for them to be really user configurable.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I understand the reasoning, but this should be consistent with the other items in the class. If it is not needed to have settable paths, then this is not the right PR to make such a change.

Copy link
Copy Markdown
Contributor Author

@himanshug himanshug Aug 30, 2017

Choose a reason for hiding this comment

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

earlier we supported druid.zk.paths.indexer.leaderLatchPath property which was being used by overlord and is obtained via IndexerZkConfig. (I'm gonna note this in the release notes)
even if I honor that property in this PR, still overlord is incompatible given that underlying algorithm for leader election changes. Given that compatibility is broken anyway, I don't see the value in making it configurable just because all the other properties in this class are such due to historical reasons. In fact, I would say any newly added property added to this class shouldn't be made configurable.

do you still feel strongly about making if configurable ? :)

@himanshug
Copy link
Copy Markdown
Contributor Author

himanshug commented Aug 24, 2017

@drcrallen thanks for checking the PR. note that intention of this PR is to only extract leader election code out of TaskMaster and DruidCoordinator classes, put it behind an interface and retain existing behavior as much as possible.
Curator based implementation is done in a way to retain existing behavior and changed only slightly to fix a bug as mentioned in #4699 (comment) .

@himanshug
Copy link
Copy Markdown
Contributor Author

@drcrallen added docs.

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

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

Looks good to me overall.

/**
* Get ID of current Leader.
*/
@Nullable
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Please add to doc when the result is null.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added

/**
* Must be called right after registerLeader(Listener).
*/
void start();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Maybe better to combine two above methods into a single method like registerListenerAndStart().

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

ok, on second thought I removed start/stop methods altogether and instead have registerListener(listener) and unregisterListener() .

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Looks nice!


/**
* Interface for supporting Overlord and Coordinator Leader Elections in TaskMaster and DruidCoordinator
* which expect appropriate implementation available in guice annotated with @IndexingService and @Coordinator
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Suggest adding more comments here that explicitly state that the values returned were true at some point during the call, and may not still be true by the time the caller reads the values.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added more docs to isLeader() and getLeader().

try {
final LeaderLatch latch = leaderLatch.get();

Participant participant = latch.getLeader();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

#3837 is still in effect here.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Fix might be outside the scope of this PR though

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

yeah, i think that would still exist.

log.makeAlert(ex, "listener becomeLeader() failed. Unable to become leader").emit();

// give others a chance to become leader.
final LeaderLatch oldLatch = createNewLeaderLatch();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This is a really interesting way of doing recursion, but seems to match what the code was doing previously.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

yeah, i pretty much took the same code

PolyBind.optionBinder(binder, Key.get(DruidLeaderSelector.class, Coordinator.class))
.addBinding(CURATOR_KEY)
.toProvider(new DruidLeaderSelectorProvider(
(zkPathsConfig) -> ZKPaths.makePath(zkPathsConfig.getCoordinatorPath(), "druid:coordinator"))
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

why is this hard coded here?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

(minor) since the path is explicitly for druid, the druid: is redundant. Can it retain the prior naming mechanism of _COORDINATOR?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

yeah, changed ... using _COORDINATOR and _OVERLORD now

@drcrallen
Copy link
Copy Markdown
Contributor

@gianm / @himanshug what does the upgrade path look like for Tranquility? How does this upgrade roll out without breaking it?

@himanshug
Copy link
Copy Markdown
Contributor Author

@drcrallen @gianm does tranquility have enough retries to handle the scenario where all overlord nodes were briefly down?

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Aug 31, 2017

@drcrallen @gianm does tranquility have enough retries to handle the scenario where all overlord nodes were briefly down?

@himanshug by default it retries basically any failure of any kind (overlord or task) for 1 minute and then gives up on that batch of events, and reports it as dropped, and will then move on to the next batch of events. So that is the behavior you would expect to see if all overlords are down for >1 minute.

@drcrallen
Copy link
Copy Markdown
Contributor

The issue here also is consistency of discovery, for Tranqulity using the HTTP connection to overlord, will it be able to discover the overlord properly without a config change

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Aug 31, 2017

The issue here also is consistency of discovery, for Tranqulity using the HTTP connection to overlord, will it be able to discover the overlord properly without a config change

I haven't looked at the code in this PR, but assuming that the overlord and also the running tasks still announce in Curator service discovery when you choose Curator-based leader election, it should all be the same to Tranquility, right @himanshug ?

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Aug 31, 2017

In general I thought one of the goals of this PR was to make it so there is no change in Curator service-discovery behavior if you stick with the Curator impl.

@drcrallen
Copy link
Copy Markdown
Contributor

It is entirely possible my brain crossed some wires when looking through the PR. Trying to keep the different ways announcements are used straight in my head was challenging

@himanshug
Copy link
Copy Markdown
Contributor Author

himanshug commented Sep 1, 2017

@gianm @drcrallen yes, this PR does not remove the announcement of overlord/coordinator leader inside "external service discovery" . that announcement is currently used by tranquility/. And, also internally by peons and coordinator to discover overlord leader, router to discovery coordinator leader. #4735 sets the stage for removing use of "external service discovery" from internal components.

wrt to tranquility or other things interacting with overlord leader, they might get temporary errors when all overlords are brought down and till at least one of them comes back up. I have updated this information in release notes section of PR description.

also, I think all the comments have been addressed at this point and PR should be merge ready.

Copy link
Copy Markdown
Contributor

@drcrallen drcrallen left a comment

Choose a reason for hiding this comment

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

@himanshug thanks a ton for your effort in getting this patch to a great state!

@drcrallen
Copy link
Copy Markdown
Contributor

I see 3 👍 's

setupServerAndCurator();
}

@Test(timeout = 5000)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think we should up this timeout, on my laptop I saw some runs that took between 5-7s, which led to some test failures

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

hmmm, earlier I did not expect it to take long but i think changing (as per the discussion #4699 (review) ) the sleep from 1 sec to a random value between 1 sec to 5 sec now can make this test take longer as this test exercises that code path. I will update the timeout to 15 secs which should be good enough.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants