Hi all, I've updated the KIP to reflect the latest discussions.
Tom, 2) Updated 4) I decided against switching to a "batch interface" and added the reasons in the Rejected Alternatives section Please take a look and let me know if you have any feedback. Thanks On Tue, Oct 6, 2020 at 9:43 AM Mickael Maison <mickael.mai...@gmail.com> wrote: > > Hi Efe, > > Thanks for the feedback. > We also need to assign replicas when adding partitions to an existing > topic. This is why I choose to use a list of partition ids. Otherwise > we'd need the number of partitions and the starting partition id. > > Let me know if you have more questions > > On Tue, Oct 6, 2020 at 2:16 AM Efe Gencer <agen...@linkedin.com.invalid> > wrote: > > > > Hi Mickael, > > > > Thanks for the KIP! > > A call to an external system, e.g. Cruise Control, in the implementation of > > the provided interface can indeed help with the initial assignment of > > partitions. > > > > I am curious why the proposed `ReplicaAssignor#assignReplicasToBrokers` > > receives a list of partition ids as opposed to the number of partitions to > > create the topic with? > > > > Would you clarify if this API is expected to be used (1) only for new > > topics or (2) also for existing topics? > > > > Best, > > Efe > > ________________________________ > > From: Mickael Maison <mickael.mai...@gmail.com> > > Sent: Thursday, October 1, 2020 9:43 AM > > To: dev <dev@kafka.apache.org> > > Subject: Re: [DISCUSS] KIP-660: Pluggable ReplicaAssignor > > > > Thanks Tom for the feedback! > > > > 1. If the data returned by the ReplicaAssignor implementation does not > > match that was requested, we'll also throw a ReplicaAssignorException > > > > 2. Good point, I'll update the KIP > > > > 3. The KIP mentions an error code associated with > > ReplicaAssignorException: REPLICA_ASSIGNOR_FAILED > > > > 4. (I'm naming your last question 4.) I spent some time looking at it. > > Initially I wanted to follow the model from the topic policies. But as > > you said, computing assignments for the whole batch may be more > > desirable and also avoids incrementally updating the cluster state. > > The logic in AdminManager is very much centered around doing 1 topic > > at a time but as far as I can tell we should be able to update it to > > compute assignments for the whole batch. > > > > I'll play a bit more with 4. and I'll update the KIP in the next few days > > > > On Mon, Sep 21, 2020 at 10:29 AM Tom Bentley <tbent...@redhat.com> wrote: > > > > > > Hi Mickael, > > > > > > A few thoughts about the ReplicaAssignor contract: > > > > > > 1. What happens if a ReplicaAssignor impl returns a Map where some > > > assignments don't meet the given replication factor? > > > 2. Fixing the signature of assignReplicasToBrokers() as you have would > > > make > > > it hard to pass extra information in the future (e.g. maybe someone comes > > > up with a use case where passing the clientId would be needed) because it > > > would require the interface be changed. If you factored all the parameters > > > into some new type then the signature could be > > > assignReplicasToBrokers(RequiredReplicaAssignment) and adding any new > > > properties to RequiredReplicaAssignment wouldn't break the contract. > > > 3. When an assignor throws RepliacAssignorException what error code will > > > be > > > returned to the client? > > > > > > Also, this sentence got me thinking: > > > > > > > If multiple topics are present in the request, AdminManager will update > > > the Cluster object so the ReplicaAssignor class has access to the up to > > > date cluster metadata. > > > > > > Previously I've looked at how we can improve Kafka's pluggable policy > > > support to pass the more of the cluster state to policy implementations. A > > > similar problem exists there, but the more cluster state you pass the > > > harder it is to incrementally change it as you iterate through the topics > > > to be created/modified. This likely isn't a problem here and now, but it > > > could limit any future changes to the pluggable assignors. Did you > > > consider > > > the alternative of the assignor just being passed a Set of assignments? > > > That means you can just pass the cluster state as it exists at the time. > > > It > > > also gives the implementation more information to work with to find more > > > optimal assignments. For example, it could perform a bin packing type > > > assignment which found a better optimum for the whole collection of topics > > > than one which was only told about all the topics in the request > > > sequentially. > > > > > > Otherwise this looks like a valuable feature to me. > > > > > > Kind regards, > > > > > > Tom > > > > > > > > > > > > > > > > > > On Fri, Sep 11, 2020 at 6:19 PM Robert Barrett <bob.barr...@confluent.io> > > > wrote: > > > > > > > Thanks Mickael, I think adding the new Exception resolves my concerns. > > > > > > > > On Thu, Sep 3, 2020 at 9:47 AM Mickael Maison <mickael.mai...@gmail.com> > > > > wrote: > > > > > > > > > Thanks Robert and Ryanne for the feedback. > > > > > > > > > > ReplicaAssignor implementations can throw an exception to indicate an > > > > > assignment can't be computed. This is already what the current round > > > > > robin assignor does. Unfortunately at the moment, there are no generic > > > > > error codes if it fails, it's either INVALID_PARTITIONS, > > > > > INVALID_REPLICATION_FACTOR or worse UNKNOWN_SERVER_ERROR. > > > > > > > > > > So I think it would be nice to introduce a new Exception/Error code to > > > > > cover any failures in the assignor and avoid UNKNOWN_SERVER_ERROR. > > > > > > > > > > I've updated the KIP accordingly, let me know if you have more > > > > > questions. > > > > > > > > > > On Fri, Aug 28, 2020 at 4:49 PM Ryanne Dolan <ryannedo...@gmail.com> > > > > > wrote: > > > > > > > > > > > > Thanks Mickael, the KIP makes sense to me, esp for cases where an > > > > > external > > > > > > system (like cruise control or an operator) knows more about the > > > > > > target > > > > > > cluster state than the broker does. > > > > > > > > > > > > Ryanne > > > > > > > > > > > > On Thu, Aug 20, 2020, 10:46 AM Mickael Maison < > > > > mickael.mai...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > Hi, > > > > > > > > > > > > > > I've created KIP-660 to make the replica assignment logic > > > > > > > pluggable. > > > > > > > > > > > > > > > > > > > > > > > https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-660%253A%2BPluggable%2BReplicaAssignor&data=02%7C01%7Cagencer%40linkedin.com%7Ca156bf97031b4100b62d08d866293434%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637371674445085863&sdata=Cz1u3y1M%2BH5dFIx%2BHkQwugN%2FqTH1ugjXaaBhbToCkDM%3D&reserved=0 > > > > > > > > > > > > > > Please take a look and let me know if you have any feedback. > > > > > > > > > > > > > > Thanks > > > > > > > > > > > > > > > >