KAFKA-5692: Change PreferredReplicaLeaderElectionCommand to use Admin…#3848
KAFKA-5692: Change PreferredReplicaLeaderElectionCommand to use Admin…#3848junrao merged 22 commits intoapache:trunkfrom
Conversation
There was a problem hiding this comment.
This is backed by List.
Change variable name accordingly.
3086bf3 to
a693d47
Compare
|
@ijuma I've rebased this PR for the |
a693d47 to
3e9408d
Compare
3e9408d to
64fcbfc
Compare
|
Rebased for merge conflict. Please could you review this when you have time @ijuma? |
|
@cmccabe can you please review this (particularly the AdminClient and protocol parts, I can take a look at the Controller bits). |
| } | ||
|
|
||
| /** Return a new future that has completed exceptionally */ | ||
| private static <T> KafkaFuture<T> exceptionalFuture(Throwable e) { |
There was a problem hiding this comment.
This is a good utility function. It makes sense to add this to function KafkaFuture.java, right? This is similar to KafkaFuture#completedFuture, which returns a future completed with the given result.
| public KafkaFuture<Void> partitionResult(TopicPartition partition) { | ||
| final Map<TopicPartition, ? extends KafkaFuture<Void>> map; | ||
| try { | ||
| map = futures.get(); |
There was a problem hiding this comment.
You are calling KafkaFuture#get here, which is blocking. Don't do that.
There was a problem hiding this comment.
@cmccabe You're right, but to fix I'll need to implement a KafkaFuture.thenCompose(Function<T, KafkaFuture<R>>) method on KafkaFuture<T>. That seems to be awkward because the API of KafkaFuture doesn't provide a public addWaiter() method, but the Function<A, KafkaFuture<B>> in the API of thenApply() is returning a KafkaFuture, not an Impl. I've pushed WIP patch, so you can see what I mean. Thoughts?
There was a problem hiding this comment.
I don't see why you need thenCompose. You already have thenApply, and you can just do something like this:
final KafkaFuture<Map<TopicPartition, Throwable>> partitionFuture = new KafkaFutureImpl<>();
rpcFuture.thenApply(new Function<Map<TopicPartition, Throwable>, Void>() {
@Override
Void apply(Map<TopicPartition, Throwable> map) {
// check map for this partition and ensure there is no error
if (not found in map) {
partitionFuture.completeExceptionally(not there);
else if (there was some error) {
partitionFuture.completeExceptionally(error);
} else {
partitionFuture.complete(null);
}
}
});
I am not opposed to adding thenCompose to KafkaFuture, but I just don't think you need it for this.
There was a problem hiding this comment.
@cmccabe that code makes no sense to me. Is it supposed to be a replacement for what's in ElectPreferredLeadersResult.partitionResult()? What's supposed to happen with your partitionFuture? It gets completed, but doesn't otherwise get used/returned/passed to anything, so nothing can handle or wait on its result. Your apply() lacks a return, if it always returns null then the result of the thenApply() is a Future<Void> that is always successful.
I think we need thenCompose() because, in effect, we want a Function that can return a Future<Void>. thenApply() forces us into returning the actual Void, but then we are having to call get() in the Function, which defeats the point. Or we can use thenApply() and return a Future<Void> from the Function, but then the result of thenApply() is a double Future<Future<Void>>.
| public KafkaFuture<Set<TopicPartition>> partitions() { | ||
| final Map<TopicPartition, ? extends KafkaFuture<Void>> map; | ||
| try { | ||
| map = futures.get(); |
| public KafkaFuture<Void> all() { | ||
| final Map<TopicPartition, ? extends KafkaFuture<Void>> map; | ||
| try { | ||
| map = futures.get(); |
| TopicPartition partition = entry.getKey(); | ||
| KafkaFutureImpl<Void> future; | ||
| if (knownPartitions) { | ||
| future = mapOfFutures.get(partition); |
There was a problem hiding this comment.
You need to handle the case where there is an incorrect server response and there is no such partition in our request. Right now, your code gets a null pointer error in this case.
| future = mapOfFutures.get(partition); | ||
| } else { | ||
| future = new KafkaFutureImpl<Void>(); | ||
| mapOfFutures.put(partition, future); |
There was a problem hiding this comment.
You can't modify this map here because it is concurrently accessed by people accessing ElectPreferredLeadersResult.
There was a problem hiding this comment.
@cmccabe there are two cases here, depending on the state of knownPartitions. We only modify mapOfFutures when knownPartitions == false and in this case it's impossible for anyone to access the map via the ElectPreferredLeadersResult because the "outer" futures won't be completed.
|
Thanks for the patch, @tombentley. There are some issues here, however. It seems like all the async APIs are actually sync, because the code to retrieve the future is calling Another issue I see is that you modify a map in the results object concurrently with users accessing the map. If you want to have your code inserting elements into the map, you need locking to do that. You also need to handle error conditions like the server sending the wrong thing, etc. |
|
Tom, there is a patch here which shows what I'm talking about: #4183 |
|
@cmccabe thanks going to the trouble of pointing out what must have seemed blindingly obvious. Your patch raises a broader question in my mind: If it's OK to just use a It was the precedents set by the existing |
|
@cmccabe a problem with your patch is that it doesn't work when the |
|
@cmccabe any thoughts about the preferred way to deal with abnormal completion of then |
It's OK. It wasn't obvious to me when I started doing async programming. But over time it became easier :)
Good catch. I think this shortcoming is why I didn't use this approach for things like createTopics.
Sorry, this got buried in my email box. But I fixed my email settings to make notifications more visible... I think @tombentley's idea of exposing the |
3a92833 to
0630c4c
Compare
|
Rebased onto current trunk. This PR now assumes that KIP-218 gets accepted and merged, since it uses |
|
Thanks for being patient, @tombentley. We're working on getting the |
0630c4c to
d6c739c
Compare
There was a problem hiding this comment.
Is it really necessary to keep around both functions? It seems like this utility function is only supposed to be used by Kafka, not by end-users.
There was a problem hiding this comment.
My intention was to retain binary compatibility. Since it's not in an internal package how would an end-user know that they weren't supposed to use this API?
There was a problem hiding this comment.
They should know because there isn't an interface annotation that declares it a public API for use outside Kafka.
There was a problem hiding this comment.
We publish javadoc for public APIs. Everything else is not public. This is a bit opaque and it would be better to have annotations as @cmccabe suggests.
There was a problem hiding this comment.
It would be nice to avoid this and similar whitespace changes
a6d1b7a to
2cd37d5
Compare
junrao
left a comment
There was a problem hiding this comment.
@tombentley : Thanks for the updated patch. A few more comments below.
| "about": "The topic partitions to elect the preferred leader of.", | ||
| "fields": [ | ||
| { "name": "Topic", "type": "string", "versions": "0+", | ||
| "about": "The name of a topic." }, |
There was a problem hiding this comment.
It seems that we should mark this field as mapKey to enforce uniqueness. @cmccabe : Would you agree?
There was a problem hiding this comment.
If I add "mapKey": true then I end up with a subclass of ImplicitLinkedHashMultiSet<TopicPartitions>, which wouldn't enforce uniqueness because it's a multiset. @cmccabe can you enlighten me? Is there a way to get a subclass of ImplicitLinkedHashSet<TopicPartitions> instead?
There was a problem hiding this comment.
@tombentley, you are correct that making this a mapKey would not inherently enforce uniqueness. However, it would make it easier to add that enforcement to the code if you need to. You could easily look up "are there any other topic entries that have the same name as this one, and set an error if so.
| "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, | ||
| { "name": "ReplicaElectionResults", "type": "[]ReplicaElectionResult", "versions": "0+", | ||
| "about": "The error code, or 0 if there was no error.", "fields": [ | ||
| { "name": "Topic", "type": "string", "versions": "0+", |
There was a problem hiding this comment.
Similar to the above, it seems that we should mark the topic field and the PartitionId field as mapKey?
There was a problem hiding this comment.
As discussed earlier, it would be useful not to use Controller.controllerContext here since it's meant to be internal to the controller.
There was a problem hiding this comment.
As mentioned earlier, this only works if the controller is the bootstrap server. Otherwise, by the time that PreferredReplicaLeaderElectionCommand runs, a new controller could have been elected.
junrao
left a comment
There was a problem hiding this comment.
@tombentley : Thanks for addressing the comments. We are pretty close. Just a few more comments below.
| val deadline = time.milliseconds() + requestTimeout | ||
|
|
||
| def electionCallback(waiting: Set[TopicPartition], | ||
| expectedLeaders: Set[ElectPreferredLeaderMetadata], |
There was a problem hiding this comment.
Hmm, the waiting param seems redundant. Could we just pass in expectedLeaders? Also, instead of creating a new type ElectPreferredLeaderMetadata, could we just pass in a Map[TP, int]?
junrao
left a comment
There was a problem hiding this comment.
@tombentley : Thanks for addressing all the comments in the PR. LGTM. About using mapKey, since currently it doesn't enforce uniqueness, it's kind of optional. So, we can just leave them out for now.
|
@tombentley : I merged the PR. The implementation has some slight modification to what's described in KIP-183. For example, the ElectPreferredLeadersRequest now has a timeout field. Could you update the KIP wiki and send a followup email to the original voting thread about the minor changes? We can then see if anyone has any additional feedback. |
|
@junrao sure, I can do that. Thanks for your help! |
* ak/trunk: MINOR: Update usage of deprecated API (apache#6146) KAFKA-4217: Add KStream.flatTransform (apache#5273) MINOR: Update Gradle to 5.1.1 (apache#6160) KAFKA-3522: Generalize Segments (apache#6170) Added quotes around the class path (apache#4469) KAFKA-7837: Ensure offline partitions are picked up as soon as possible when shrinking ISR (apache#6202) MINOR: In the MetadataResponse schema, ignorable should be a boolean KAFKA-7838: Log leader and follower end offsets when shrinking ISR (apache#6168) KAFKA-5692: Change PreferredReplicaLeaderElectionCommand to use Admin… (apache#3848) MINOR: clarify why suppress can sometimes drop tombstones (apache#6195) MINOR: Upgrade ducktape to 0.7.5 (apache#6197) MINOR: Improve IntegrationTestUtils documentation (apache#5664) MINOR: upgrade to jdk8 8u202 KAFKA-7693; Fix SequenceNumber overflow in producer (apache#5989) KAFKA-7692; Fix ProducerStateManager SequenceNumber overflow (apache#5990) MINOR: update copyright year in the NOTICE file. (apache#6196) KAFKA-7793: Improve the Trogdor command line. (apache#6133)
apache#3848) See also KIP-183. This implements the following algorithm: AdminClient sends ElectPreferredLeadersRequest. KafakApis receives ElectPreferredLeadersRequest and delegates to ReplicaManager.electPreferredLeaders() ReplicaManager delegates to KafkaController.electPreferredLeaders() KafkaController adds a PreferredReplicaLeaderElection to the EventManager, ReplicaManager.electPreferredLeaders()'s callback uses the delayedElectPreferredReplicasPurgatory to wait for the results of the election to appear in the metadata cache. If there are no results because of errors, or because the preferred leaders are already leading the partitions then a response is returned immediately. In the EventManager work thread the preferred leader is elected as follows: The EventManager runs PreferredReplicaLeaderElection.process() process() calls KafkaController.onPreferredReplicaElectionWithResults() KafkaController.onPreferredReplicaElectionWithResults() calls the PartitionStateMachine.handleStateChangesWithResults() to perform the election (asynchronously the PSM will send LeaderAndIsrRequest to the new and old leaders and UpdateMetadataRequest to all brokers) then invokes the callback. Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jun Rao <junrao@gmail.com>
…Client
See also KIP-183.
The contribution is my original work and I license the work to the project under the project's open source license.
This implements the following algorithm:
ReplicaManager.electPreferredLeaders()
delayedElectPreferredReplicasPurgatory to wait for the results of the
election to appear in the metadata cache. If there are no results
because of errors, or because the preferred leaders are already leading
the partitions then a response is returned immediately.
In the EventManager work thread the preferred leader is elected as follows:
calls the PartitionStateMachine.handleStateChangesWithResults() to
perform the election (asynchronously the PSM will send LeaderAndIsrRequest
to the new and old leaders and UpdateMetadataRequest to all brokers)
then invokes the callback.
Note: the change in parameter type for CollectionUtils.groupDataByTopic().
This makes sense because the AdminClient APIs use Collection consistently,
rather than List or Set. If binary compatiblity is a consideration the old
version should be kept, delegating to the new version.
I had to add PartitionStateMachine.handleStateChangesWithResults()
in order to be able to process a set of state changes in the
PartitionStateMachine and get back individual results.
At the same time I noticed that all callers of existing handleStateChange()
were destructuring a TopicAndPartition that they already had in order
to call handleStateChange(), and that handleStateChange() immediately
instantiated a new TopicAndPartition. Since TopicAndPartition is immutable
this is pointless, so I refactored it. handleStateChange() also now returns
any exception it caught, which is necessary for handleStateChangesWithResults()