On Mon, Sep 4, 2017, at 04:54, Tom Bentley wrote: > The KIP has been adopted after a successful vote.
Thanks for working on this, Tom. It's a nice improvement. > > Unfortunately I've discovered that there's an annoying detail in the > handling of the case that electPreferredLeaders() is called with a null > partitions argument. As discussed with Ewen, this is supposed to mean > "all > partitions", but we don't know all the partitions in the AdminClient, yet > we have to return a ElectPreferredLeadersResults instance, supposedly > with > the partitions as keys. > > We could handle this by passing a KafkaFuture<Map<TopicPartition, > KafkaFuture<Void>>> to the ctor of ElectPreferredLeadersResults, instead > of > an extant Map<TopicPartition, KafkaFuture<Void>> (the API of > ElectPreferredLeadersResults would not change). In the case that the > partitions argument was not null this future will already be completed. > In > the case where partitions argument was null this future will be completed > when we have a response from which we discover the partitions; in the > meantime the AdminClient can carry on being used for other calls. So in > the > normal case there's not really a problem. > > The problem comes where there's an exception *before we get the > response*, > that means we still don't know the partitions to populate the map with. > In > practice this would mean that an exception could propagate out of > ElectPreferredLeadersResults.values() rather than when the map was > accessed > element-wise. Since we control the API of ElectPreferredLeadersResults we > could document that values() (and consequently all()) could throw,. We > could even use checked exceptions, though since the exception would only > happen in the case that the partitions argument was null that would feel > rather heavy-handed to me. > > Another alternative would be to block in > AdminClient.electPreferredLeaders() > in the case that the partitions argument was null, and if there was an > error let the exception propagate out of electPreferredLeaders() > directly. > > Sorry about having to ask about this once people have already voted, but > what do people think about these options? I think we need to be very careful to keep the APIs asynchronous all the time. Having electPreferredLeaders() or values() sometimes block might superficially seem reasonable, but it destroys the usefulness of the API for true async programming. Basically the async user is forced to put the call into a thread pool in case it decides to block. And they cannot do nice things like chain Futures. Why does there need to be a map at all in the API? Why not just have something like this: ElectPreferredLeadersResults { /** * Get the result of the election for the given TopicPartition. * If there was not an election triggered for the given TopicPartition, the * future will complete with an error. */ public KafkaFuture<Void> partitionResult(TopicPartition topicPartition); /** * Get the topic partitions on which we attempted to trigger an election. * This tracks attempts, not successes. A partition will appear in this result * even if the election was not successfully triggered. */ public KafkaFuture<Collection<TopicPartition>> partitions(); /** * Return a future which gives an error result if we fail for any partition. */ public KafkaFuture<Void> all(); } We can fill in all this information when we actually know it. In some cases that will be later than others. But all the calls can immediately return a KafkaFuture, not block or throw an exception. best, Colin > > Thanks, > > Tom > > On 30 August 2017 at 16:55, Tom Bentley <t.j.bent...@gmail.com> wrote: > > > I've updated in the KIP. > > > > Thanks, > > > > Tom > > > > On 30 August 2017 at 16:42, Ismael Juma <ism...@juma.me.uk> wrote: > > > >> If you agree with the change, yes, please rename. It's OK to make changes > >> after the VOTE thread starts. In cases where some people have already > >> voted, it's recommended to mention the changes in the VOTE thread as a > >> heads up. Generally, we don't restart the vote unless the changes are > >> significant. > >> > >> Ismael > >> > >> On Wed, Aug 30, 2017 at 4:26 PM, Tom Bentley <t.j.bent...@gmail.com> > >> wrote: > >> > >> > Hi Ismael, > >> > > >> > I agree that `electPreferredReplicaLeader` is a mouthful and am happy to > >> > change it to `electPreferredLeaders`. I'd rename the correspond request > >> and > >> > response similarly. > >> > > >> > Should I rename it in the KIP now, even though I initiated a VOTE thread > >> > yesterday? > >> > > >> > Cheers, > >> > > >> > Tom > >> > > >> > On 30 August 2017 at 16:01, Ismael Juma <ism...@juma.me.uk> wrote: > >> > > >> > > Hi Tom, > >> > > > >> > > Thanks for the KIP, it's a useful one. I find the proposed method name > >> > > `electPreferredReplicaLeader` a little hard to read. It seems that a > >> > small > >> > > change would make it clearer: `electPreferredReplicaAsLeader`. The > >> next > >> > > point is that this is a batch API, so it should ideally be plural like > >> > the > >> > > other AdminClient methods. Maybe `electPreferredReplicasAsLeaders`, > >> but > >> > > that's quite a mouthful. Maybe we should shorten it to > >> > > `electPreferredLeaders`. Thoughts? > >> > > > >> > > Ismael > >> > > > >> > > On Wed, Aug 2, 2017 at 6:34 PM, Tom Bentley <t.j.bent...@gmail.com> > >> > wrote: > >> > > > >> > > > In a similar vein to KIP-179 I've created KIP-183 ( > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-183+-+Change+ > >> > > > PreferredReplicaLeaderElectionCommand+to+use+AdminClient) > >> > > > which is about deprecating the --zookeeper option to > >> > > > kafka-preferred-replica-election.sh and replacing it with an option > >> > > which > >> > > > would use a new AdminClient-based API. > >> > > > > >> > > > As it stands the KIP is focussed on simply moving the existing > >> > > > functionality behind the AdminClient. > >> > > > > >> > > > I'd be grateful for any feedback people may have on this. > >> > > > > >> > > > Thanks, > >> > > > > >> > > > Tom > >> > > > > >> > > > >> > > >> > > > >