You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Colin McCabe <cm...@apache.org> on 2019/05/09 05:22:40 UTC

[VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Hi all,

I'd like to start the vote for KIP-455: Create an Administrative API for Replica Reassignment.  I think this KIP is important since it will unlock many follow-on improvements to Kafka reassignment (see the "Future work" section, plus a lot of the other discussions we've had recently about reassignment).  It also furthers the important KIP-4 goal of removing direct access to ZK.

I made a few changes based on the discussion in the [DISCUSS] thread.  As Robert suggested, I removed the need to explicitly cancel a reassignment for a partition before setting up a different reassignment for that specific partition.  I also simplified the API a bit by adding a PartitionReassignment class which is used by both the alter and list APIs.

I modified the proposal so that we now deprecate the old znode-based API rather than removing it completely.  That should give external rebalancing tools some time to transition to the new API.

To clarify a question Viktor asked, I added a note that the kafka-reassign-partitions.sh will now use a --bootstrap-server argument to contact the admin APIs. 

thanks,
Colin

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Robert Barrett <bo...@confluent.io>.
+1 (non-binding)

Thanks for the KIP, Colin!

On Thu, May 9, 2019 at 8:27 AM Colin McCabe <cm...@apache.org> wrote:

> Hi Viktor,
>
> There is a jira -- KAFKA-8345.  The PR is not quite ready yet, but
> hopefully soon :)
>
> best,
> Colin
>
> On Thu, May 9, 2019, at 01:13, Viktor Somogyi-Vass wrote:
> > +1 (non-binding)
> >
> > Thanks Colin, this is great stuff. Does a jira (or maybe even a PR :) )
> for
> > this exist yet?
> >
> > Viktor
> >
> > On Thu, May 9, 2019 at 7:23 AM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi all,
> > >
> > > I'd like to start the vote for KIP-455: Create an Administrative API
> for
> > > Replica Reassignment.  I think this KIP is important since it will
> unlock
> > > many follow-on improvements to Kafka reassignment (see the "Future
> work"
> > > section, plus a lot of the other discussions we've had recently about
> > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > direct access to ZK.
> > >
> > > I made a few changes based on the discussion in the [DISCUSS] thread.
> As
> > > Robert suggested, I removed the need to explicitly cancel a
> reassignment
> > > for a partition before setting up a different reassignment for that
> > > specific partition.  I also simplified the API a bit by adding a
> > > PartitionReassignment class which is used by both the alter and list
> APIs.
> > >
> > > I modified the proposal so that we now deprecate the old znode-based
> API
> > > rather than removing it completely.  That should give external
> rebalancing
> > > tools some time to transition to the new API.
> > >
> > > To clarify a question Viktor asked, I added a note that the
> > > kafka-reassign-partitions.sh will now use a --bootstrap-server
> argument to
> > > contact the admin APIs.
> > >
> > > thanks,
> > > Colin
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi Viktor,

There is a jira -- KAFKA-8345.  The PR is not quite ready yet, but hopefully soon :)

best,
Colin

On Thu, May 9, 2019, at 01:13, Viktor Somogyi-Vass wrote:
> +1 (non-binding)
> 
> Thanks Colin, this is great stuff. Does a jira (or maybe even a PR :) ) for
> this exist yet?
> 
> Viktor
> 
> On Thu, May 9, 2019 at 7:23 AM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi all,
> >
> > I'd like to start the vote for KIP-455: Create an Administrative API for
> > Replica Reassignment.  I think this KIP is important since it will unlock
> > many follow-on improvements to Kafka reassignment (see the "Future work"
> > section, plus a lot of the other discussions we've had recently about
> > reassignment).  It also furthers the important KIP-4 goal of removing
> > direct access to ZK.
> >
> > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > Robert suggested, I removed the need to explicitly cancel a reassignment
> > for a partition before setting up a different reassignment for that
> > specific partition.  I also simplified the API a bit by adding a
> > PartitionReassignment class which is used by both the alter and list APIs.
> >
> > I modified the proposal so that we now deprecate the old znode-based API
> > rather than removing it completely.  That should give external rebalancing
> > tools some time to transition to the new API.
> >
> > To clarify a question Viktor asked, I added a note that the
> > kafka-reassign-partitions.sh will now use a --bootstrap-server argument to
> > contact the admin APIs.
> >
> > thanks,
> > Colin
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Viktor Somogyi-Vass <vi...@gmail.com>.
+1 (non-binding)

Thanks Colin, this is great stuff. Does a jira (or maybe even a PR :) ) for
this exist yet?

Viktor

On Thu, May 9, 2019 at 7:23 AM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> I'd like to start the vote for KIP-455: Create an Administrative API for
> Replica Reassignment.  I think this KIP is important since it will unlock
> many follow-on improvements to Kafka reassignment (see the "Future work"
> section, plus a lot of the other discussions we've had recently about
> reassignment).  It also furthers the important KIP-4 goal of removing
> direct access to ZK.
>
> I made a few changes based on the discussion in the [DISCUSS] thread.  As
> Robert suggested, I removed the need to explicitly cancel a reassignment
> for a partition before setting up a different reassignment for that
> specific partition.  I also simplified the API a bit by adding a
> PartitionReassignment class which is used by both the alter and list APIs.
>
> I modified the proposal so that we now deprecate the old znode-based API
> rather than removing it completely.  That should give external rebalancing
> tools some time to transition to the new API.
>
> To clarify a question Viktor asked, I added a note that the
> kafka-reassign-partitions.sh will now use a --bootstrap-server argument to
> contact the admin APIs.
>
> thanks,
> Colin
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Good news-- we are finally getting support for this in kafka-reassign-partitions.sh in Kafka 2.6.  I updated the KIP to state that the controller changes were made in 2.4 and the command line changes will appear in 2.6.  Previously it just said that the KIP was implemented in 2.4.

Just as a quick note, I changed the section describing the --list tool slightly to reflect the fact that it now outputs the replicas list, adding replica list, and removing replica list.  This puts it  more in line with what the API outputs.  Also, the --create-cancellation flag has been renamed to --cancel.  It now cancels reassignments for all the partitions supplied in the JSON file.  I think this will be easier to use and less complex than the previously proposed two-step process of generating a cancellation plan and applying it.

cheers,
Colin


On Fri, Oct 25, 2019, at 01:04, Stanislav Kozlovski wrote:
> Hello all,
> 
> To best shape up KIP-455 before 2.4 releases, we had some very minor
> last-minute improvements which I'd like to share for the record.
> 
> 1) MINOR: Check against empty replicas in AlterPartitionReassignments (
> https://github.com/apache/kafka/commit/78e7c90e90efa18b2a5b298e49154834d8d5bf67
> )
> Added validation against passing in a set of empty replicas (e.g []) to the
> alter API. We now properly raise a InvalidReplicaAssignmentException :)
> 
> 2) MINOR: ListPartitionReassignmentsResponse should not be entirely failed
> when a topic-partition does not exist (
> https://github.com/apache/kafka/commit/fa2a9f09e4042f821d7373e2d9e01b21aede775a
> )
> The List API would fail the whole request if one topic partition didn't
> exist. We now simply ignore that partition in the response
> 
> 3) Iterate on the NewPartitionReassignment interface
> We had left the design of that interface to the implementation and did 
> not
> discuss it in the voting thread, as it is a very trivial class that 
> should
> be used only for passing in a parameter to the Alter API.
> Regardless, we iterated on it in PR discussions and ended up with the
> following interface, described in the KIP. (
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=112820260&selectedPageVersions=42&selectedPageVersions=41
> )
> 
> I've done my best to keep the KIP page up to date with the latest
> information.
> 
> Thanks again to everybody who helped discuss, vote on, review and implement
> this KIP. :)
> 
> On Wed, Aug 28, 2019 at 8:47 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi all,
> >
> > After some discussion with Jun and Stan, we decided that we should bump
> > the version of the topics znode from 1 to 2.  The bump is backwards
> > compatible (older brokers can read the v2 znode).  I have updated the KIP.
> >
> > best,
> > Colin
> >
> >
> > On Thu, Aug 8, 2019, at 11:09, Colin McCabe wrote:
> > > Hi Koushik,
> > >
> > > The vote for this KIP already passed.
> > >
> > > See https://www.mail-archive.com/dev@kafka.apache.org/msg99636.html
> > >
> > > best,
> > > Colin
> > >
> > > On Thu, Aug 8, 2019, at 10:50, Koushik Chitta wrote:
> > > > Thanks Colin, George.   Can we restart the voting for this KIP.
> > > >
> > > > Thanks,
> > > > Koushik
> > > >
> > > > -----Original Message-----
> > > > From: Colin McCabe <cm...@apache.org>
> > > > Sent: Wednesday, August 7, 2019 5:17 PM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica
> > > > Reassignment
> > > >
> > > > On Wed, Aug 7, 2019, at 15:41, George Li wrote:
> > > > > This email seemed to get lost in the dev email server.  Resending.
> > > > >
> > > > >
> > > > > On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li
> > > > > <sq...@yahoo.com> wrote:
> > > > >
> > > > >
> > > > > The pending reassignments partitions would be reported as URP (Under
> > > > > Replicated Partitions).  or maybe reported as a separate metrics of
> > > > > RURP (Reassignment URP) since now we can derived from the new
> > > > > AddingReplicas. An alert could be triggered based on this.
> > > > >
> > > >
> > > > Hi George,
> > > >
> > > > I agree that this would be a great idea for follow up work.  Check out
> > > > KIP-352, which discusses creating a such a metric. :)
> > > >
> > > > >
> > > > >
> > > > > It would be nice if ListPartitionReassignmentResult could return the
> > > > > "elapsed time/duration" of the current pending reassignments, the
> > > > > calling client can flag those current long running reassignments and
> > > > > alert.  However, what I would be interested is probably the total #
> > of
> > > > > pending reassignments because I will submit reassignments in
> > batches,
> > > > > e.g. 50 reassignments per batch.  If the pending reassignments # is
> > > > > below that per batch #, submit more new reassignments = (per_batch_#
> > -
> > > > > pending_#).
> > > > >
> > > >
> > > > It is definitely useful to know what reassignments exist.  If you call
> > > > ListPartitionReassignments, you can count how many results you get, in
> > > > order to implement a policy like that.
> > > >
> > > > I'm not sure if knowing how long reassignments have been in progress
> > > > will be important or not.  I think we should give people some time to
> > > > try out the new APIs and see what could be improved based on their
> > > > experience.
> > > >
> > > > >
> > > > >
> > > > > It seems currently, the ReplicaFetcher threads could quite easily
> > crash
> > > > > because of some exceptions. e.g. Java Out Of Memory, and would just
> > > > > remain dead (jstack to dump threads to check the # of running
> > > > > ReplicaFetcher threads) without getting restarted automatically, so
> > > > > needs to bounce the broker.  It would be nice to make the
> > > > > ReplicaFetcher more robust/resilient of catching more exceptions,
> > and
> > > > > if crashed, get restarted after some time.
> > > > >
> > > >
> > > > This has definitely been an issue in the past, I agree.  Thankfully,
> > we
> > > > recently did improve the robustness of the ReplicaFetcher.  Check out
> > > > "KIP-461: Improve Replica Fetcher behavior at handling partition
> > > > failure."
> > > >
> > > > cheers,
> > > > Colin
> > > >
> > > > >
> > > > >
> > > > > Thanks,
> > > > >
> > > > > George
> > > > >
> > > > >
> > > > >
> > > > > On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote:
> > > > > > Hi Koushik,
> > > > > >
> > > > > > Thanks for the idea.  This KIP is already pretty big, so I think
> > we'll have to consider ideas like this in follow-on KIPs.
> > > > > >
> > > > > > In general, figuring out what's wrong with replication is a pretty
> > tough problem.  If we had an API for this, we'd probably want it to be
> > unified, and not specific to reassigning partitions.
> > > > > >
> > > > > > regards,
> > > > > > Colin
> > > > > >
> > > > > >
> > > > > > On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > > > > > > Hey Colin,
> > > > > > >
> > > > > > > Can the ListPartitionReassignmentsResult include the status of
> > the
> > > > > > > current reassignment progress of each partition? A reassignment
> > can be
> > > > > > > in progress for different reasons and the status can give the
> > option to
> > > > > > > alter the current reassignment.
> > > > > > >
> > > > > > > Example -  A leaderISRRequest of a new assigned replicas can be
> > > > > > > ignored/errored because of a storage exception.  And
> > reassignment batch
> > > > > > > will be waiting indefinitely for the new assigned replicas to be
> > in
> > > > > > > sync with the leader of the partition.
> > > > > > >             Showing the status will give an option to alter the
> > affected
> > > > > > > partitions and allow the batch to complete reassignment.
> > > > > > >
> > > > > > > OAR = {1, 2, 3} and RAR = {4,5,6}
> > > > > > >
> > > > > > >      AR                 leader/isr
> > > > > > >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>
> > LeaderISRRequest
> > > > > > > was lost/skipped for 5 and the reassignment operation will be
> > waiting
> > > > > > > indefinitely for the 5 to be insync.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Koushik
> > > > > > >
> > > > > > > -----Original Message-----
> > > > > > > From: Jun Rao <ju...@confluent.io>
> > > > > > > Sent: Friday, August 2, 2019 10:04 AM
> > > > > > > To: dev <de...@kafka.apache.org>
> > > > > > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for
> > Replica
> > > > > > > Reassignment
> > > > > > >
> > > > > > > Hi, Colin,
> > > > > > >
> > > > > > > First, since we are changing the format of LeaderAndIsrRequest,
> > which
> > > > > > > is an inter broker request, it seems that we will need IBP
> > during
> > > > > > > rolling upgrade. Could we add that to the compatibility section?
> > > > > > >
> > > > > > > Regarding UnsupportedVersionException, even without ZK node
> > version
> > > > > > > bump, we probably want to only use the new ZK value fields after
> > all
> > > > > > > brokers have been upgraded to the new binary. Otherwise, the
> > > > > > > reassignment task may not be completed if the controller changes
> > to a
> > > > > > > broker still on the old binary.
> > > > > > > IBP is one way to achieve that. The main thing is that we need
> > some way
> > > > > > > for the controller to deal with the new ZK fields. Dealing with
> > the
> > > > > > > additional ZK node version bump seems a small thing on top of
> > that?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > > > >
> > > > > > > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > > > > > > Hi, Colin,
> > > > > > > > >
> > > > > > > > > 10. Sounds good.
> > > > > > > > >
> > > > > > > > > 13. Our current convention is to bump up the version of ZK
> > value if
> > > > > > > > > there is any format change. For example, we have bumped up
> > the
> > > > > > > > > version of the value in /brokers/ids/nnn multiple times and
> > all of
> > > > > > > > > those changes are compatible (just adding new fields). This
> > has the
> > > > > > > > > slight benefit that it makes it clear there is a format
> > change.
> > > > > > > > > Rolling upgrades and downgrades can still be supported with
> > the
> > > > > > > > > version bump. For example, if you
> > > > > > > > downgrade
> > > > > > > > > from a compatible change, you can leave the new format in ZK
> > and the
> > > > > > > > > old code will only pick up fields relevant to the old
> > version.
> > > > > > > > > Upgrade will
> > > > > > > > be
> > > > > > > > > controlled by inter broker protocol.
> > > > > > > >
> > > > > > > > Hmm.  If we bump that ZK node version, we will need a new
> > inter-broker
> > > > > > > > protocol version.  We also need to return
> > UnsupportedVersionException
> > > > > > > > from the alterPartitionReassignments and
> > listPartitionReassignments
> > > > > > > > APIs when the IBP is too low.  This sounds doable, although we
> > might
> > > > > > > > need a release note that upgrading the IBP is necessary to
> > allow
> > > > > > > > reassignment operations after an upgrade.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <
> > cmccabe@apache.org> wrote:
> > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > >
> > > > > > > > > > Thanks for taking another look at this.
> > > > > > > > > >
> > > > > > > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > > > > > > Hi, Stan,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the explanation.
> > > > > > > > > > >
> > > > > > > > > > > 10. If those new fields in LeaderAndIsr are only needed
> > for
> > > > > > > > > > > future
> > > > > > > > work,
> > > > > > > > > > > perhaps they should be added when we do the future work
> > instead
> > > > > > > > > > > of
> > > > > > > > now?
> > > > > > > > > >
> > > > > > > > > > I think this ties in with one of the big goals of this
> > KIP, making
> > > > > > > > > > it possible to distinguish reassigning replicas from
> > normal replicas.
> > > > > > > > This is
> > > > > > > > > > the key to follow-on work like being able to ensure that
> > > > > > > > > > partitions
> > > > > > > > with a
> > > > > > > > > > reassignment don't get falsely flagged as under-replicated
> > in the
> > > > > > > > metrics,
> > > > > > > > > > or implementing reassignment quotas that don't
> > accidentally affect
> > > > > > > > normal
> > > > > > > > > > replication traffic when a replica falls out of the ISR.
> > > > > > > > > >
> > > > > > > > > > For these follow-on improvements, we need to have that
> > information
> > > > > > > > > > in LeaderAndIsrRequest.  We could add the information in a
> > > > > > > > > > follow-on KIP,
> > > > > > > > of
> > > > > > > > > > course, but then all the improvements are blocked on that
> > > > > > > > > > follow-on
> > > > > > > > KIP.
> > > > > > > > > > That would slow things down for all of the downstream KIPs
> > that
> > > > > > > > > > are
> > > > > > > > blocked
> > > > > > > > > > on this.
> > > > > > > > > >
> > > > > > > > > > Also, to keep things consistent, I think it would be best
> > if the
> > > > > > > > format of
> > > > > > > > > > the data in the LeaderAndIsrRequest matched the format of
> > the data
> > > > > > > > > > in ZooKeeper.  Since we're deciding on the ZK format in
> > this KIP,
> > > > > > > > > > I think
> > > > > > > > it
> > > > > > > > > > makes sense to also decide on the format in the
> > LeaderAndIsrRequest.
> > > > > > > > > >
> > > > > > > > > > > > > Should we include those two fields in UpdateMetadata
> > and
> > > > > > > > potentially
> > > > > > > > > > > > > Metadata requests too?
> > > > > > > > > >
> > > > > > > > > > We had some discussion earlier about how metadata
> > responses to
> > > > > > > > > > clients
> > > > > > > > are
> > > > > > > > > > getting too large, in part because they include a lot of
> > > > > > > > > > information
> > > > > > > > that
> > > > > > > > > > most clients don't need (such as the ISR).  I think
> > reassignment
> > > > > > > > > > information definitely falls in the category of something
> > a client
> > > > > > > > doesn't
> > > > > > > > > > need to know, so we shouldn't include it.
> > > > > > > > > >
> > > > > > > > > > A program like CruiseControl, or the command-line
> > reassignment
> > > > > > > > > > program, just wants to get the most up-to-date information
> > about
> > > > > > > > > > the state of reassigning partitions.  The MetadataRequest
> > API
> > > > > > > > > > wouldn't deliver that, because there are inherently delays
> > in how
> > > > > > > > > > we propagate metadata to brokers.  That's why the
> > > > > > > > > > ListPartitionReassignments API is a better
> > > > > > > > choice
> > > > > > > > > > for those programs.  So I think if we added this
> > information to
> > > > > > > > > > the MetadataResponse, nobody would actually use it, and it
> > would
> > > > > > > > > > just use
> > > > > > > > up
> > > > > > > > > > more bandwidth.
> > > > > > > > > >
> > > > > > > > > > Of course, we can always revisit this later if we find a
> > scenario
> > > > > > > > where a
> > > > > > > > > > producer or consumer would actually care about this.  But
> > I think
> > > > > > > > > > we
> > > > > > > > should
> > > > > > > > > > default to not adding stuff to the metadata response if we
> > don't
> > > > > > > > > > have a good use case in mind.
> > > > > > > > > >
> > > > > > > > > > > > > 11. "If a new reassignment is issued during an
> > on-going one,
> > > > > > > > > > > > > we
> > > > > > > > > > cancel the
> > > > > > > > > > > > > current one by emptying out both AR and RR,
> > constructing
> > > > > > > > > > > > > them
> > > > > > > > from
> > > > > > > > > > (the
> > > > > > > > > > > > > updated from the last-reassignment) R and TR, and
> > starting
> > > > > > > > anew." In
> > > > > > > > > > this
> > > > > > > > > > > > > case, it seems that the controller needs to issue a
> > > > > > > > > > > > > StopReplica
> > > > > > > > > > request to
> > > > > > > > > > > > > remove those unneeded replicas.
> > > > > > > > > >
> > > > > > > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > > > > > > >
> > > > > > > > > > > > > 12. "Essentially, once a cancellation is called we
> > subtract
> > > > > > > > > > > > > AR
> > > > > > > > from
> > > > > > > > > > R,
> > > > > > > > > > > > > empty out both AR and RR, and send LeaderAndIsr
> > requests to
> > > > > > > > cancel
> > > > > > > > > > the
> > > > > > > > > > > > > replica movements that have not yet completed."
> > Similar to
> > > > > > > > > > > > > the
> > > > > > > > > > above, it
> > > > > > > > > > > > > seems the controller needs to issue a StopReplica
> > request to
> > > > > > > > remove
> > > > > > > > > > those
> > > > > > > > > > > > > unneeded replicas.
> > > > > > > > > >
> > > > > > > > > > Right.  Let's add this.
> > > > > > > > > >
> > > > > > > > > > > > > 13. Since we changed the format of the
> > topics/[topic] zNode,
> > > > > > > > should
> > > > > > > > > > we bump
> > > > > > > > > > > > > up the version number in the json value?
> > > > > > > > > >
> > > > > > > > > > The change to the zNode is backwards compatible, though.
> > Older
> > > > > > > > > > brokers will continue to work, but just ignore the new
> > fields.  If
> > > > > > > > > > we bump that version number, then downgrades will require
> > hand-editing zookeeper.
> > > > > > > > (Of
> > > > > > > > > > course downgrade isn't officially supported, but it would
> > be nice
> > > > > > > > > > not
> > > > > > > > to
> > > > > > > > > > break it if we don't need to...)  Changing the version
> > number
> > > > > > > > > > would
> > > > > > > > also
> > > > > > > > > > create problems during a rolling upgrade.
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe
> > > > > > > > > > > > > <cmccabe@apache.org
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > With three non-binding +1 votes from Viktor
> > Somogyi-Vass,
> > > > > > > > Robert
> > > > > > > > > > > > Barrett,
> > > > > > > > > > > > > > and George Li, and 3 binding +1 votes from Gwen
> > Shapira,
> > > > > > > > > > > > > > Jason
> > > > > > > > > > > > Gustafson,
> > > > > > > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett
> > wrote:
> > > > > > > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > > > > > > sql_consulting@yahoo.com
> > > > > > > > > > > > > > .invalid>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM
> > PDT, Gwen
> > > > > > > > Shapira <
> > > > > > > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for
> > working
> > > > > > > > through
> > > > > > > > > > all
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > questions, edge cases, requests and
> > alternatives. We
> > > > > > > > > > > > > > > > ended
> > > > > > > > up
> > > > > > > > > > with
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > great protocol.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason
> > Gustafson <
> > > > > > > > > > > > jason@confluent.io>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 Thanks for the KIP. Really looking
> > forward to this!
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > -Jason
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe
> > <
> > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote
> > to
> > > > > > > > > > > > > > > > > > reflect
> > > > > > > > the
> > > > > > > > > > fact
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > we've
> > > > > > > > > > > > > > > > > > made significant changes.  The new vote
> > will go
> > > > > > > > > > > > > > > > > > for 3
> > > > > > > > days
> > > > > > > > > > as
> > > > > > > > > > > > > > usual.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav
> > > > > > > > > > > > > > > > > > Kozlovski
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We have further iterated on the KIP in
> > the
> > > > > > > > accompanying
> > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > and I'd like to propose we resume the
> > vote.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > > > > > > - we will store reassignment information
> > in the
> > > > > > > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > > > > > > - we will internally use two collections
> > to
> > > > > > > > represent a
> > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > > > > > > LeaderAndIsr has
> > > > > > > > > > > > been
> > > > > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > > > > > > - the Alter API will still use the
> > "targetReplicas"
> > > > > > > > > > > > collection,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > List API will now return three separate
> > > > > > > > > > > > > > > > > > > collections
> > > > > > > > - the
> > > > > > > > > > > > full
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > set,
> > > > > > > > > > > > > > > > > > > the replicas we are adding as part of
> > this
> > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > > > > > > and the replicas we are removing
> > > > > > > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > > > > > > - cancellation of a reassignment now
> > means a
> > > > > > > > > > > > > > > > > > > proper
> > > > > > > > > > rollback
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > assignment to its original state prior
> > to the
> > > > > > > > > > > > > > > > > > > API
> > > > > > > > call
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > https://cwik
> > > > > > > > i.apache.org
> > %2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > > > > > >
> > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > > > > > > C01%7Ckchitta%40microsoft.com
> > %7C09235110291f453cb18608d7176b8505%7C72f
> > > > > > > >
> > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > > > > > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin
> > McCabe <
> > > > > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks for taking a look.  I am
> > working on
> > > > > > > > > > > > > > > > > > > > getting
> > > > > > > > a PR
> > > > > > > > > > > > done
> > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.
> > Then
> > > > > > > > > > > > > > > > > > > > we'll
> > > > > > > > > > finish up
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > vote.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George
> > Li wrote:
> > > > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >  Great! Looking forward to these
> > features.    +1
> > > > > > > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > What is the estimated timeline to
> > have this
> > > > > > > > > > implemented?
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > > > > > is needed in the implementation of
> > > > > > > > > > > > > > > > > > > > > cancelling
> > > > > > > > > > > > > > reassignments,  I
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >    On Thursday, May 16, 2019,
> > 9:48:56 AM
> > > > > > > > > > > > > > > > > > > > > PDT,
> > > > > > > > Colin
> > > > > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment
> > of
> > > > > > > > individual
> > > > > > > > > > > > > > partitions to
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > cancelled.  I think it's very
> > important for
> > > > > > > > > > > > > > > > > > > > > these
> > > > > > > > > > > > > operations
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > be at
> > > > > > > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34,
> > George Li wrote:
> > > > > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It
> > has very
> > > > > > > > > > > > > > > > > > > > > > good
> > > > > > > > > > > > > improvements
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > One question, looks like the KIP
> > includes
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > Cancellation
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > individual pending reassignments
> > as well
> > > > > > > > > > > > > > > > > > > > > > when
> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest
> > has
> > > > > > > > > > > > > > > > > > > > > > empty
> > > > > > > > > > replicas
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > topic/partition. Will you also be
> > > > > > > > > > > > > > > > > > > > > > implementing
> > > > > > > > the
> > > > > > > > > > the
> > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?
> >   If yes,
> > > > > > > > it
> > > > > > > > > > will
> > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > KIP-236
> > > > > > > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > > has PR already) trivial, since the
> > cancel
> > > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > pending
> > > > > > > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > > > > > > one just needs to do a
> > > > > > > > > > ListPartitionRessignmentRequest,
> > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > > > > > > empty replicas for all those
> > > > > > > > > > > > > > > > > > > > > > topic/partitions
> > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > one
> > AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >    On Friday, May 10, 2019,
> > 8:44:31 PM
> > > > > > > > > > > > > > > > > > > > > > PDT,
> > > > > > > > Colin
> > > > > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34,
> > Colin
> > > > > > > > > > > > > > > > > > > > > > McCabe
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43,
> > Jason
> > > > > > > > Gustafson
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > I think storing reassignment
> > state at
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > partition
> > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > > > > > > and I also agree that replicas
> > should
> > > > > > > > > > understand
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > there is
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > reassignment in progress. This
> > makes
> > > > > > > > KIP-352 a
> > > > > > > > > > > > > trivial
> > > > > > > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > example. The only doubt I have
> > is
> > > > > > > > > > > > > > > > > > > > > > > > whether
> > > > > > > > the
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > isr
> > > > > > > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > > > > > right place to store the target
> > > > > > > > reassignment.
> > > > > > > > > > It
> > > > > > > > > > > > is a
> > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > odd
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > > > > > > target assignment in a
> > separate place
> > > > > > > > > > > > > > > > > > > > > > > > from
> > > > > > > > the
> > > > > > > > > > > > > current
> > > > > > > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > > > > > > assume the thinking is
> > probably that
> > > > > > > > although
> > > > > > > > > > the
> > > > > > > > > > > > > > current
> > > > > > > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > > > > > > probably be in the leader and
> > isr
> > > > > > > > > > > > > > > > > > > > > > > > znode as
> > > > > > > > > > well, it
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > hard to
> > > > > > > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > > > > > > state in a compatible way. Is
> > that right?
> > > > > > > > But
> > > > > > > > > > if we
> > > > > > > > > > > > > > have no
> > > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > > > > > > the assignment znode, do you
> > see a
> > > > > > > > downside to
> > > > > > > > > > > > > storing
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > That's a good point -- it's
> > probably
> > > > > > > > > > > > > > > > > > > > > > > better
> > > > > > > > to
> > > > > > > > > > keep
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > > > assignment in the same znode as
> > the
> > > > > > > > > > > > > > > > > > > > > > > current
> > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > consistency.  I'll change the
> > KIP.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I took another look at this, and I
> > think
> > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > should
> > > > > > > > > > > > stick
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > initial proposal of putting the
> > > > > > > > > > > > > > > > > > > > > > reassignment
> > > > > > > > state
> > > > > > > > > > into
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > > > > > > The
> > > > > > > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > > > > > > because we'll want to bump the
> > leader
> > > > > > > > > > > > > > > > > > > > > > epoch
> > > > > > > > for the
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > changing the reassignment state,
> > and the
> > > > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > epoch
> > > > > > > > > > > > > > resides
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > znode anyway.  I agree there is
> > some
> > > > > > > > inconsistency
> > > > > > > > > > > > here,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > so be
> > > > > > > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > > > > > > if we were to greenfield these
> > zookeeper
> > > > > > > > > > > > > > > > > > > > > > data
> > > > > > > > > > > > structures,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > might
> > > > > > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > > > > > it differently, but the proposed
> > scheme
> > > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > work
> > > > > > > > > > fine
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 1. Should
> > > > > > > > > > > > > > > > > > > > > > > > `alterPartitionReassignments`
> > be
> > > > > > > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > > > > > > It's the current assignment
> > we're
> > > > > > > > > > > > > > > > > > > > > > > > altering,
> > > > > > > > > > right?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > That's fair.
> > AlterPartitionAssigments
> > > > > > > > > > > > > > > > > > > > > > > reads
> > > > > > > > a
> > > > > > > > > > little
> > > > > > > > > > > > > > > > better, and
> > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API
> > name in
> > > > > > > > > > > > > > > > > > > > > > +the
> > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 2. Does this change affect the
> > > > > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > > > API? In
> > > > > > > > > > > > other
> > > > > > > > > > > > > > > > words,
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > > > > > aware of reassignments? If so,
> > then we
> > > > > > > > probably
> > > > > > > > > > > > need
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The
> > only
> > > > > > > > alternative I
> > > > > > > > > > can
> > > > > > > > > > > > > > think of
> > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > > > > > > represent the replica set in
> > the
> > > > > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > > > > > request
> > > > > > > > > > > > as
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > union
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > current and target replicas,
> > but I
> > > > > > > > > > > > > > > > > > > > > > > > can't
> > > > > > > > think
> > > > > > > > > > of
> > > > > > > > > > > > any
> > > > > > > > > > > > > > > > benefit
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > > > > > > reassignments. Note that if we
> > did
> > > > > > > > > > > > > > > > > > > > > > > > this, we
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > wouldn't
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > separate API to list
> > reassignments.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I thought about this a bit...
> > and I
> > > > > > > > > > > > > > > > > > > > > > > think on
> > > > > > > > > > balance,
> > > > > > > > > > > > > > you're
> > > > > > > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > should keep this information
> > together
> > > > > > > > > > > > > > > > > > > > > > > with
> > > > > > > > the
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > > > > > > nodes, and offline replicas, and
> > that
> > > > > > > > > > information is
> > > > > > > > > > > > > > > > available in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > > > > >  However, I do think in order to
> > do
> > > > > > > > > > > > > > > > > > > > > > > this,
> > > > > > > > we'll
> > > > > > > > > > need
> > > > > > > > > > > > a
> > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > MetadataRequest that specifiies
> > "only
> > > > > > > > > > > > > > > > > > > > > > > show me
> > > > > > > > > > > > > reassigning
> > > > > > > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I revisited this, and I think we
> > should
> > > > > > > > > > > > > > > > > > > > > > stick
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > > > > original
> > > > > > > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > > > > > > ListPartitionReassignments
> > > > > > > > > > > > > > API.
> > > > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > > > > really is no use case where the
> > Producer
> > > > > > > > > > > > > > > > > > > > > > or
> > > > > > > > > > Consumer
> > > > > > > > > > > > > needs
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > about a reassignment.  They should
> > just be
> > > > > > > > notified
> > > > > > > > > > > > when
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > > > > > partitions changes, which doesn't
> > require
> > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  The
> > Admin
> > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > only
> > > > > > > > > > cares
> > > > > > > > > > > > if
> > > > > > > > > > > > > > > > someone
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > managing the reassignment.  So
> > adding this
> > > > > > > > state
> > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > > MetadataResponse adds overhead for
> > no real
> > > > > > > > > > benefit.  In
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > common
> > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > where there is no ongoing
> > reassignment, it
> > > > > > > > would
> > > > > > > > > > be 4
> > > > > > > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > In general, I think we have a
> > problem of
> > > > > > > > > > oversharing in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we
> > 10x or
> > > > > > > > > > > > > > > > > > > > > > 100x
> > > > > > > > the
> > > > > > > > > > number
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > > > > we support, we'll need to get
> > stricter
> > > > > > > > > > > > > > > > > > > > > > about
> > > > > > > > giving
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > information they actually need,
> > about the
> > > > > > > > > > partitions
> > > > > > > > > > > > they
> > > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > > > > > > about.  Reassignment state clearly
> > falls
> > > > > > > > > > > > > > > > > > > > > > in the
> > > > > > > > > > > > category
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > state
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > isn't needed by clients (except
> > very
> > > > > > > > specialized
> > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Another important consideration
> > here is
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > someone
> > > > > > > > > > > > > > managing
> > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > ongoing reassignment wants the
> > most
> > > > > > > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > > > > > > information,
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > be found on the controller.
> > Therefore
> > > > > > > > > > > > > > > > > > > > > > adding
> > > > > > > > this
> > > > > > > > > > > > state
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > > > > > > or describeTopics, which could
> > contact any
> > > > > > > > node in
> > > > > > > > > > the
> > > > > > > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Finally, adding this to listTopics
> > or
> > > > > > > > > > describeTopics
> > > > > > > > > > > > > feels
> > > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > > > > > > API.  It's an extra boolean which
> > > > > > > > > > > > > > > > > > > > > > interacts
> > > > > > > > with
> > > > > > > > > > other
> > > > > > > > > > > > > > extra
> > > > > > > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > > > > > > like "show internal", etc. in
> > weird ways.
> > > > > > > > > > > > > > > > > > > > > > I
> > > > > > > > think
> > > > > > > > > > a
> > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > API
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 3. As replicas come into sync,
> > they
> > > > > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > join
> > > > > > > > > > the
> > > > > > > > > > > > > ISR.
> > > > > > > > > > > > > > > > Will we
> > > > > > > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > > > > > > target replicas joining the
> > ISR before
> > > > > > > > taking
> > > > > > > > > > the
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > > > > > > replicas set? Also, I assume
> > that
> > > > > > > > > > > > > > > > > > > > > > > > target
> > > > > > > > > > replicas
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > still be
> > > > > > > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > We'll take a replica out of the
> > target
> > > > > > > > replicas
> > > > > > > > > > set
> > > > > > > > > > > > as
> > > > > > > > > > > > > > soon
> > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me
> > clarify
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 4. Probably useful to mention
> > > > > > > > > > > > > > > > > > > > > > > > permissions
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > new
> > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Good point.  I think
> > > > > > > > alterPartitionAssignments
> > > > > > > > > > should
> > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest
> > permissions
> > > > > > > > > > > > > > > > > > > > > > > will be
> > > > > > > > > > > > > unchanged.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30
> > AM Gwen
> > > > > > > > Shapira <
> > > > > > > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > > > > > > Looks great, and will be
> > awesome to
> > > > > > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > this new
> > > > > > > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23
> > PM
> > > > > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > McCabe
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote
> > for KIP-455:
> > > > > > > > > > Create
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I
> > think
> > > > > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > KIP is
> > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > > > > > > many follow-on
> > improvements to
> > > > > > > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > (see
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > > > > > > section, plus a lot of the
> > other
> > > > > > > > > > discussions
> > > > > > > > > > > > > we've
> > > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > > > > > > reassignment).  It also
> > furthers
> > > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > important
> > > > > > > > > > > > > > KIP-4
> > > > > > > > > > > > > > > > goal
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > I made a few changes based
> > on the
> > > > > > > > > > discussion in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > > > > > > Robert suggested, I
> > removed the
> > > > > > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > > > for a partition before
> > setting up
> > > > > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > different
> > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > > specific partition.  I
> > also
> > > > > > > > > > > > > > > > > > > > > > > > > > simplified
> > > > > > > > the
> > > > > > > > > > API
> > > > > > > > > > > > a
> > > > > > > > > > > > > > bit by
> > > > > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > > PartitionReassignment
> > class which
> > > > > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > used
> > > > > > > > > > by
> > > > > > > > > > > > both
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > alter
> > > > > > > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > I modified the proposal so
> > that we
> > > > > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > > > > > deprecate
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > old
> > > > > > > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > > > > > > rather than removing it
> > completely.
> > > > > > > > That
> > > > > > > > > > > > should
> > > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > > > > > > tools some time to
> > transition to
> > > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > new
> > > > > > > > > > API.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > To clarify a question
> > Viktor
> > > > > > > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > > > > > > added a
> > > > > > > > > > > > > note
> > > > > > > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > kafka-reassign-partitions.sh will
> > > > > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > use a
> > > > > > > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product
> > Manager |
> > > > > > > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > Follow us:
> > > > > > > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > > > > > > https:
> > > > > > > > > > > > >
> > %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > > > > > > 40microsoft.com
> > %7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > > > > > >
> > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > > > > > >
> > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > > > > > > served=0>
> > > > > > > > > > > > > > |
> > > > > > > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > > > > > > <
> > https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnam06.safelinks.protection&amp;data=02%7C01%7Ckchitta%40microsoft.com%7C3e662dd1d60949f2ff6808d71b95e68f%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637008202957720437&amp;sdata=vdDj2GPhjVP5TQj1B4%2BoCjGU6cgDqGVYOEnqaMurz68%3D&amp;reserved=0
> > .
> > > > > > > > > > > > > > > > > > > > > > > > >
> > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > > > > > > nfluent.io
> > %2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com
> > %7C092351102
> > > > > > > > > > > > > > > > > > > > > > > > >
> > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > > > > > >
> > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > > > > > >
> > 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > > > > > >
> > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter |
> > blog
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 
> 
> -- 
> Best,
> Stanislav
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Stanislav Kozlovski <st...@confluent.io>.
Hello all,

To best shape up KIP-455 before 2.4 releases, we had some very minor
last-minute improvements which I'd like to share for the record.

1) MINOR: Check against empty replicas in AlterPartitionReassignments (
https://github.com/apache/kafka/commit/78e7c90e90efa18b2a5b298e49154834d8d5bf67
)
Added validation against passing in a set of empty replicas (e.g []) to the
alter API. We now properly raise a InvalidReplicaAssignmentException :)

2) MINOR: ListPartitionReassignmentsResponse should not be entirely failed
when a topic-partition does not exist (
https://github.com/apache/kafka/commit/fa2a9f09e4042f821d7373e2d9e01b21aede775a
)
The List API would fail the whole request if one topic partition didn't
exist. We now simply ignore that partition in the response

3) Iterate on the NewPartitionReassignment interface
We had left the design of that interface to the implementation and did not
discuss it in the voting thread, as it is a very trivial class that should
be used only for passing in a parameter to the Alter API.
Regardless, we iterated on it in PR discussions and ended up with the
following interface, described in the KIP. (
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=112820260&selectedPageVersions=42&selectedPageVersions=41
)

I've done my best to keep the KIP page up to date with the latest
information.

Thanks again to everybody who helped discuss, vote on, review and implement
this KIP. :)

On Wed, Aug 28, 2019 at 8:47 PM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> After some discussion with Jun and Stan, we decided that we should bump
> the version of the topics znode from 1 to 2.  The bump is backwards
> compatible (older brokers can read the v2 znode).  I have updated the KIP.
>
> best,
> Colin
>
>
> On Thu, Aug 8, 2019, at 11:09, Colin McCabe wrote:
> > Hi Koushik,
> >
> > The vote for this KIP already passed.
> >
> > See https://www.mail-archive.com/dev@kafka.apache.org/msg99636.html
> >
> > best,
> > Colin
> >
> > On Thu, Aug 8, 2019, at 10:50, Koushik Chitta wrote:
> > > Thanks Colin, George.   Can we restart the voting for this KIP.
> > >
> > > Thanks,
> > > Koushik
> > >
> > > -----Original Message-----
> > > From: Colin McCabe <cm...@apache.org>
> > > Sent: Wednesday, August 7, 2019 5:17 PM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica
> > > Reassignment
> > >
> > > On Wed, Aug 7, 2019, at 15:41, George Li wrote:
> > > > This email seemed to get lost in the dev email server.  Resending.
> > > >
> > > >
> > > > On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li
> > > > <sq...@yahoo.com> wrote:
> > > >
> > > >
> > > > The pending reassignments partitions would be reported as URP (Under
> > > > Replicated Partitions).  or maybe reported as a separate metrics of
> > > > RURP (Reassignment URP) since now we can derived from the new
> > > > AddingReplicas. An alert could be triggered based on this.
> > > >
> > >
> > > Hi George,
> > >
> > > I agree that this would be a great idea for follow up work.  Check out
> > > KIP-352, which discusses creating a such a metric. :)
> > >
> > > >
> > > >
> > > > It would be nice if ListPartitionReassignmentResult could return the
> > > > "elapsed time/duration" of the current pending reassignments, the
> > > > calling client can flag those current long running reassignments and
> > > > alert.  However, what I would be interested is probably the total #
> of
> > > > pending reassignments because I will submit reassignments in
> batches,
> > > > e.g. 50 reassignments per batch.  If the pending reassignments # is
> > > > below that per batch #, submit more new reassignments = (per_batch_#
> -
> > > > pending_#).
> > > >
> > >
> > > It is definitely useful to know what reassignments exist.  If you call
> > > ListPartitionReassignments, you can count how many results you get, in
> > > order to implement a policy like that.
> > >
> > > I'm not sure if knowing how long reassignments have been in progress
> > > will be important or not.  I think we should give people some time to
> > > try out the new APIs and see what could be improved based on their
> > > experience.
> > >
> > > >
> > > >
> > > > It seems currently, the ReplicaFetcher threads could quite easily
> crash
> > > > because of some exceptions. e.g. Java Out Of Memory, and would just
> > > > remain dead (jstack to dump threads to check the # of running
> > > > ReplicaFetcher threads) without getting restarted automatically, so
> > > > needs to bounce the broker.  It would be nice to make the
> > > > ReplicaFetcher more robust/resilient of catching more exceptions,
> and
> > > > if crashed, get restarted after some time.
> > > >
> > >
> > > This has definitely been an issue in the past, I agree.  Thankfully,
> we
> > > recently did improve the robustness of the ReplicaFetcher.  Check out
> > > "KIP-461: Improve Replica Fetcher behavior at handling partition
> > > failure."
> > >
> > > cheers,
> > > Colin
> > >
> > > >
> > > >
> > > > Thanks,
> > > >
> > > > George
> > > >
> > > >
> > > >
> > > > On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote:
> > > > > Hi Koushik,
> > > > >
> > > > > Thanks for the idea.  This KIP is already pretty big, so I think
> we'll have to consider ideas like this in follow-on KIPs.
> > > > >
> > > > > In general, figuring out what's wrong with replication is a pretty
> tough problem.  If we had an API for this, we'd probably want it to be
> unified, and not specific to reassigning partitions.
> > > > >
> > > > > regards,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > > > > > Hey Colin,
> > > > > >
> > > > > > Can the ListPartitionReassignmentsResult include the status of
> the
> > > > > > current reassignment progress of each partition? A reassignment
> can be
> > > > > > in progress for different reasons and the status can give the
> option to
> > > > > > alter the current reassignment.
> > > > > >
> > > > > > Example -  A leaderISRRequest of a new assigned replicas can be
> > > > > > ignored/errored because of a storage exception.  And
> reassignment batch
> > > > > > will be waiting indefinitely for the new assigned replicas to be
> in
> > > > > > sync with the leader of the partition.
> > > > > >             Showing the status will give an option to alter the
> affected
> > > > > > partitions and allow the batch to complete reassignment.
> > > > > >
> > > > > > OAR = {1, 2, 3} and RAR = {4,5,6}
> > > > > >
> > > > > >      AR                 leader/isr
> > > > > >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>
> LeaderISRRequest
> > > > > > was lost/skipped for 5 and the reassignment operation will be
> waiting
> > > > > > indefinitely for the 5 to be insync.
> > > > > >
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Koushik
> > > > > >
> > > > > > -----Original Message-----
> > > > > > From: Jun Rao <ju...@confluent.io>
> > > > > > Sent: Friday, August 2, 2019 10:04 AM
> > > > > > To: dev <de...@kafka.apache.org>
> > > > > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for
> Replica
> > > > > > Reassignment
> > > > > >
> > > > > > Hi, Colin,
> > > > > >
> > > > > > First, since we are changing the format of LeaderAndIsrRequest,
> which
> > > > > > is an inter broker request, it seems that we will need IBP
> during
> > > > > > rolling upgrade. Could we add that to the compatibility section?
> > > > > >
> > > > > > Regarding UnsupportedVersionException, even without ZK node
> version
> > > > > > bump, we probably want to only use the new ZK value fields after
> all
> > > > > > brokers have been upgraded to the new binary. Otherwise, the
> > > > > > reassignment task may not be completed if the controller changes
> to a
> > > > > > broker still on the old binary.
> > > > > > IBP is one way to achieve that. The main thing is that we need
> some way
> > > > > > for the controller to deal with the new ZK fields. Dealing with
> the
> > > > > > additional ZK node version bump seems a small thing on top of
> that?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > > > >
> > > > > > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > > > > > Hi, Colin,
> > > > > > > >
> > > > > > > > 10. Sounds good.
> > > > > > > >
> > > > > > > > 13. Our current convention is to bump up the version of ZK
> value if
> > > > > > > > there is any format change. For example, we have bumped up
> the
> > > > > > > > version of the value in /brokers/ids/nnn multiple times and
> all of
> > > > > > > > those changes are compatible (just adding new fields). This
> has the
> > > > > > > > slight benefit that it makes it clear there is a format
> change.
> > > > > > > > Rolling upgrades and downgrades can still be supported with
> the
> > > > > > > > version bump. For example, if you
> > > > > > > downgrade
> > > > > > > > from a compatible change, you can leave the new format in ZK
> and the
> > > > > > > > old code will only pick up fields relevant to the old
> version.
> > > > > > > > Upgrade will
> > > > > > > be
> > > > > > > > controlled by inter broker protocol.
> > > > > > >
> > > > > > > Hmm.  If we bump that ZK node version, we will need a new
> inter-broker
> > > > > > > protocol version.  We also need to return
> UnsupportedVersionException
> > > > > > > from the alterPartitionReassignments and
> listPartitionReassignments
> > > > > > > APIs when the IBP is too low.  This sounds doable, although we
> might
> > > > > > > need a release note that upgrading the IBP is necessary to
> allow
> > > > > > > reassignment operations after an upgrade.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <
> cmccabe@apache.org> wrote:
> > > > > > > >
> > > > > > > > > Hi Jun,
> > > > > > > > >
> > > > > > > > > Thanks for taking another look at this.
> > > > > > > > >
> > > > > > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > > > > > Hi, Stan,
> > > > > > > > > >
> > > > > > > > > > Thanks for the explanation.
> > > > > > > > > >
> > > > > > > > > > 10. If those new fields in LeaderAndIsr are only needed
> for
> > > > > > > > > > future
> > > > > > > work,
> > > > > > > > > > perhaps they should be added when we do the future work
> instead
> > > > > > > > > > of
> > > > > > > now?
> > > > > > > > >
> > > > > > > > > I think this ties in with one of the big goals of this
> KIP, making
> > > > > > > > > it possible to distinguish reassigning replicas from
> normal replicas.
> > > > > > > This is
> > > > > > > > > the key to follow-on work like being able to ensure that
> > > > > > > > > partitions
> > > > > > > with a
> > > > > > > > > reassignment don't get falsely flagged as under-replicated
> in the
> > > > > > > metrics,
> > > > > > > > > or implementing reassignment quotas that don't
> accidentally affect
> > > > > > > normal
> > > > > > > > > replication traffic when a replica falls out of the ISR.
> > > > > > > > >
> > > > > > > > > For these follow-on improvements, we need to have that
> information
> > > > > > > > > in LeaderAndIsrRequest.  We could add the information in a
> > > > > > > > > follow-on KIP,
> > > > > > > of
> > > > > > > > > course, but then all the improvements are blocked on that
> > > > > > > > > follow-on
> > > > > > > KIP.
> > > > > > > > > That would slow things down for all of the downstream KIPs
> that
> > > > > > > > > are
> > > > > > > blocked
> > > > > > > > > on this.
> > > > > > > > >
> > > > > > > > > Also, to keep things consistent, I think it would be best
> if the
> > > > > > > format of
> > > > > > > > > the data in the LeaderAndIsrRequest matched the format of
> the data
> > > > > > > > > in ZooKeeper.  Since we're deciding on the ZK format in
> this KIP,
> > > > > > > > > I think
> > > > > > > it
> > > > > > > > > makes sense to also decide on the format in the
> LeaderAndIsrRequest.
> > > > > > > > >
> > > > > > > > > > > > Should we include those two fields in UpdateMetadata
> and
> > > > > > > potentially
> > > > > > > > > > > > Metadata requests too?
> > > > > > > > >
> > > > > > > > > We had some discussion earlier about how metadata
> responses to
> > > > > > > > > clients
> > > > > > > are
> > > > > > > > > getting too large, in part because they include a lot of
> > > > > > > > > information
> > > > > > > that
> > > > > > > > > most clients don't need (such as the ISR).  I think
> reassignment
> > > > > > > > > information definitely falls in the category of something
> a client
> > > > > > > doesn't
> > > > > > > > > need to know, so we shouldn't include it.
> > > > > > > > >
> > > > > > > > > A program like CruiseControl, or the command-line
> reassignment
> > > > > > > > > program, just wants to get the most up-to-date information
> about
> > > > > > > > > the state of reassigning partitions.  The MetadataRequest
> API
> > > > > > > > > wouldn't deliver that, because there are inherently delays
> in how
> > > > > > > > > we propagate metadata to brokers.  That's why the
> > > > > > > > > ListPartitionReassignments API is a better
> > > > > > > choice
> > > > > > > > > for those programs.  So I think if we added this
> information to
> > > > > > > > > the MetadataResponse, nobody would actually use it, and it
> would
> > > > > > > > > just use
> > > > > > > up
> > > > > > > > > more bandwidth.
> > > > > > > > >
> > > > > > > > > Of course, we can always revisit this later if we find a
> scenario
> > > > > > > where a
> > > > > > > > > producer or consumer would actually care about this.  But
> I think
> > > > > > > > > we
> > > > > > > should
> > > > > > > > > default to not adding stuff to the metadata response if we
> don't
> > > > > > > > > have a good use case in mind.
> > > > > > > > >
> > > > > > > > > > > > 11. "If a new reassignment is issued during an
> on-going one,
> > > > > > > > > > > > we
> > > > > > > > > cancel the
> > > > > > > > > > > > current one by emptying out both AR and RR,
> constructing
> > > > > > > > > > > > them
> > > > > > > from
> > > > > > > > > (the
> > > > > > > > > > > > updated from the last-reassignment) R and TR, and
> starting
> > > > > > > anew." In
> > > > > > > > > this
> > > > > > > > > > > > case, it seems that the controller needs to issue a
> > > > > > > > > > > > StopReplica
> > > > > > > > > request to
> > > > > > > > > > > > remove those unneeded replicas.
> > > > > > > > >
> > > > > > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > > > > > >
> > > > > > > > > > > > 12. "Essentially, once a cancellation is called we
> subtract
> > > > > > > > > > > > AR
> > > > > > > from
> > > > > > > > > R,
> > > > > > > > > > > > empty out both AR and RR, and send LeaderAndIsr
> requests to
> > > > > > > cancel
> > > > > > > > > the
> > > > > > > > > > > > replica movements that have not yet completed."
> Similar to
> > > > > > > > > > > > the
> > > > > > > > > above, it
> > > > > > > > > > > > seems the controller needs to issue a StopReplica
> request to
> > > > > > > remove
> > > > > > > > > those
> > > > > > > > > > > > unneeded replicas.
> > > > > > > > >
> > > > > > > > > Right.  Let's add this.
> > > > > > > > >
> > > > > > > > > > > > 13. Since we changed the format of the
> topics/[topic] zNode,
> > > > > > > should
> > > > > > > > > we bump
> > > > > > > > > > > > up the version number in the json value?
> > > > > > > > >
> > > > > > > > > The change to the zNode is backwards compatible, though.
> Older
> > > > > > > > > brokers will continue to work, but just ignore the new
> fields.  If
> > > > > > > > > we bump that version number, then downgrades will require
> hand-editing zookeeper.
> > > > > > > (Of
> > > > > > > > > course downgrade isn't officially supported, but it would
> be nice
> > > > > > > > > not
> > > > > > > to
> > > > > > > > > break it if we don't need to...)  Changing the version
> number
> > > > > > > > > would
> > > > > > > also
> > > > > > > > > create problems during a rolling upgrade.
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe
> > > > > > > > > > > > <cmccabe@apache.org
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > >
> > > > > > > > > > > > > With three non-binding +1 votes from Viktor
> Somogyi-Vass,
> > > > > > > Robert
> > > > > > > > > > > Barrett,
> > > > > > > > > > > > > and George Li, and 3 binding +1 votes from Gwen
> Shapira,
> > > > > > > > > > > > > Jason
> > > > > > > > > > > Gustafson,
> > > > > > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett
> wrote:
> > > > > > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > > > > > sql_consulting@yahoo.com
> > > > > > > > > > > > > .invalid>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM
> PDT, Gwen
> > > > > > > Shapira <
> > > > > > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for
> working
> > > > > > > through
> > > > > > > > > all
> > > > > > > > > > > the
> > > > > > > > > > > > > > > questions, edge cases, requests and
> alternatives. We
> > > > > > > > > > > > > > > ended
> > > > > > > up
> > > > > > > > > with
> > > > > > > > > > > a
> > > > > > > > > > > > > > > great protocol.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason
> Gustafson <
> > > > > > > > > > > jason@confluent.io>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 Thanks for the KIP. Really looking
> forward to this!
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > -Jason
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe
> <
> > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote
> to
> > > > > > > > > > > > > > > > > reflect
> > > > > > > the
> > > > > > > > > fact
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > we've
> > > > > > > > > > > > > > > > > made significant changes.  The new vote
> will go
> > > > > > > > > > > > > > > > > for 3
> > > > > > > days
> > > > > > > > > as
> > > > > > > > > > > > > usual.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav
> > > > > > > > > > > > > > > > > Kozlovski
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > We have further iterated on the KIP in
> the
> > > > > > > accompanying
> > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > and I'd like to propose we resume the
> vote.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > > > > > - we will store reassignment information
> in the
> > > > > > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > > > > > - we will internally use two collections
> to
> > > > > > > represent a
> > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > > > > > LeaderAndIsr has
> > > > > > > > > > > been
> > > > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > > > > > - the Alter API will still use the
> "targetReplicas"
> > > > > > > > > > > collection,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > List API will now return three separate
> > > > > > > > > > > > > > > > > > collections
> > > > > > > - the
> > > > > > > > > > > full
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > set,
> > > > > > > > > > > > > > > > > > the replicas we are adding as part of
> this
> > > > > > > reassignment
> > > > > > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > > > > > and the replicas we are removing
> > > > > > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > > > > > - cancellation of a reassignment now
> means a
> > > > > > > > > > > > > > > > > > proper
> > > > > > > > > rollback
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > assignment to its original state prior
> to the
> > > > > > > > > > > > > > > > > > API
> > > > > > > call
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > https://cwik
> > > > > > > i.apache.org
> %2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > > > > >
> an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > > > > > C01%7Ckchitta%40microsoft.com
> %7C09235110291f453cb18608d7176b8505%7C72f
> > > > > > >
> 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > > > > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin
> McCabe <
> > > > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for taking a look.  I am
> working on
> > > > > > > > > > > > > > > > > > > getting
> > > > > > > a PR
> > > > > > > > > > > done
> > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.
> Then
> > > > > > > > > > > > > > > > > > > we'll
> > > > > > > > > finish up
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > vote.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George
> Li wrote:
> > > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >  Great! Looking forward to these
> features.    +1
> > > > > > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > What is the estimated timeline to
> have this
> > > > > > > > > implemented?
> > > > > > > > > > > > If
> > > > > > > > > > > > > any
> > > > > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > > > > is needed in the implementation of
> > > > > > > > > > > > > > > > > > > > cancelling
> > > > > > > > > > > > > reassignments,  I
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >    On Thursday, May 16, 2019,
> 9:48:56 AM
> > > > > > > > > > > > > > > > > > > > PDT,
> > > > > > > Colin
> > > > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment
> of
> > > > > > > individual
> > > > > > > > > > > > > partitions to
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > cancelled.  I think it's very
> important for
> > > > > > > > > > > > > > > > > > > > these
> > > > > > > > > > > > operations
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > be at
> > > > > > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34,
> George Li wrote:
> > > > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It
> has very
> > > > > > > > > > > > > > > > > > > > > good
> > > > > > > > > > > > improvements
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > One question, looks like the KIP
> includes
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > Cancellation
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > individual pending reassignments
> as well
> > > > > > > > > > > > > > > > > > > > > when
> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest
> has
> > > > > > > > > > > > > > > > > > > > > empty
> > > > > > > > > replicas
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > topic/partition. Will you also be
> > > > > > > > > > > > > > > > > > > > > implementing
> > > > > > > the
> > > > > > > > > the
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?
>   If yes,
> > > > > > > it
> > > > > > > > > will
> > > > > > > > > > > > make
> > > > > > > > > > > > > > > KIP-236
> > > > > > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > has PR already) trivial, since the
> cancel
> > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > pending
> > > > > > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > > > > > one just needs to do a
> > > > > > > > > ListPartitionRessignmentRequest,
> > > > > > > > > > > > > then
> > > > > > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > > > > > empty replicas for all those
> > > > > > > > > > > > > > > > > > > > > topic/partitions
> > > > > > > in
> > > > > > > > > > > > > > > > > > > > > one
> AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >    On Friday, May 10, 2019,
> 8:44:31 PM
> > > > > > > > > > > > > > > > > > > > > PDT,
> > > > > > > Colin
> > > > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34,
> Colin
> > > > > > > > > > > > > > > > > > > > > McCabe
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43,
> Jason
> > > > > > > Gustafson
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I think storing reassignment
> state at
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > partition
> > > > > > > > > > > > > level
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > > > > > and I also agree that replicas
> should
> > > > > > > > > understand
> > > > > > > > > > > that
> > > > > > > > > > > > > > > there is
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > reassignment in progress. This
> makes
> > > > > > > KIP-352 a
> > > > > > > > > > > > trivial
> > > > > > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > example. The only doubt I have
> is
> > > > > > > > > > > > > > > > > > > > > > > whether
> > > > > > > the
> > > > > > > > > > > leader
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > isr
> > > > > > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > > > > right place to store the target
> > > > > > > reassignment.
> > > > > > > > > It
> > > > > > > > > > > is a
> > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > odd
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > > > > > target assignment in a
> separate place
> > > > > > > > > > > > > > > > > > > > > > > from
> > > > > > > the
> > > > > > > > > > > > current
> > > > > > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > > > > > assume the thinking is
> probably that
> > > > > > > although
> > > > > > > > > the
> > > > > > > > > > > > > current
> > > > > > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > > > > > probably be in the leader and
> isr
> > > > > > > > > > > > > > > > > > > > > > > znode as
> > > > > > > > > well, it
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > hard to
> > > > > > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > > > > > state in a compatible way. Is
> that right?
> > > > > > > But
> > > > > > > > > if we
> > > > > > > > > > > > > have no
> > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > > > > > the assignment znode, do you
> see a
> > > > > > > downside to
> > > > > > > > > > > > storing
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > That's a good point -- it's
> probably
> > > > > > > > > > > > > > > > > > > > > > better
> > > > > > > to
> > > > > > > > > keep
> > > > > > > > > > > the
> > > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > > assignment in the same znode as
> the
> > > > > > > > > > > > > > > > > > > > > > current
> > > > > > > > > > > assignment,
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > consistency.  I'll change the
> KIP.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I took another look at this, and I
> think
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > should
> > > > > > > > > > > stick
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > initial proposal of putting the
> > > > > > > > > > > > > > > > > > > > > reassignment
> > > > > > > state
> > > > > > > > > into
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > > > > > The
> > > > > > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > > > > > because we'll want to bump the
> leader
> > > > > > > > > > > > > > > > > > > > > epoch
> > > > > > > for the
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > changing the reassignment state,
> and the
> > > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > > > > resides
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > znode anyway.  I agree there is
> some
> > > > > > > inconsistency
> > > > > > > > > > > here,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > so be
> > > > > > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > > > > > if we were to greenfield these
> zookeeper
> > > > > > > > > > > > > > > > > > > > > data
> > > > > > > > > > > structures,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > might
> > > > > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > > > > it differently, but the proposed
> scheme
> > > > > > > > > > > > > > > > > > > > > will
> > > > > > > work
> > > > > > > > > fine
> > > > > > > > > > > > and
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 1. Should
> > > > > > > > > > > > > > > > > > > > > > > `alterPartitionReassignments`
> be
> > > > > > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > > > > > It's the current assignment
> we're
> > > > > > > > > > > > > > > > > > > > > > > altering,
> > > > > > > > > right?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > That's fair.
> AlterPartitionAssigments
> > > > > > > > > > > > > > > > > > > > > > reads
> > > > > > > a
> > > > > > > > > little
> > > > > > > > > > > > > > > better, and
> > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API
> name in
> > > > > > > > > > > > > > > > > > > > > +the
> > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 2. Does this change affect the
> > > > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > > API? In
> > > > > > > > > > > other
> > > > > > > > > > > > > > > words,
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > > > > aware of reassignments? If so,
> then we
> > > > > > > probably
> > > > > > > > > > > need
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The
> only
> > > > > > > alternative I
> > > > > > > > > can
> > > > > > > > > > > > > think of
> > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > > > > > represent the replica set in
> the
> > > > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > > > > request
> > > > > > > > > > > as
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > union
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > current and target replicas,
> but I
> > > > > > > > > > > > > > > > > > > > > > > can't
> > > > > > > think
> > > > > > > > > of
> > > > > > > > > > > any
> > > > > > > > > > > > > > > benefit
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > > > > > reassignments. Note that if we
> did
> > > > > > > > > > > > > > > > > > > > > > > this, we
> > > > > > > > > > > probably
> > > > > > > > > > > > > > > wouldn't
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > separate API to list
> reassignments.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I thought about this a bit...
> and I
> > > > > > > > > > > > > > > > > > > > > > think on
> > > > > > > > > balance,
> > > > > > > > > > > > > you're
> > > > > > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > should keep this information
> together
> > > > > > > > > > > > > > > > > > > > > > with
> > > > > > > the
> > > > > > > > > > > replica
> > > > > > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > > > > > nodes, and offline replicas, and
> that
> > > > > > > > > information is
> > > > > > > > > > > > > > > available in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > > > >  However, I do think in order to
> do
> > > > > > > > > > > > > > > > > > > > > > this,
> > > > > > > we'll
> > > > > > > > > need
> > > > > > > > > > > a
> > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > MetadataRequest that specifiies
> "only
> > > > > > > > > > > > > > > > > > > > > > show me
> > > > > > > > > > > > reassigning
> > > > > > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I revisited this, and I think we
> should
> > > > > > > > > > > > > > > > > > > > > stick
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > > original
> > > > > > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > > > > > ListPartitionReassignments
> > > > > > > > > > > > > API.
> > > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > > > really is no use case where the
> Producer
> > > > > > > > > > > > > > > > > > > > > or
> > > > > > > > > Consumer
> > > > > > > > > > > > needs
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > about a reassignment.  They should
> just be
> > > > > > > notified
> > > > > > > > > > > when
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > > > > partitions changes, which doesn't
> require
> > > > > > > changes
> > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  The
> Admin
> > > > > > > > > > > > > > > > > > > > > client
> > > > > > > only
> > > > > > > > > cares
> > > > > > > > > > > if
> > > > > > > > > > > > > > > someone
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > managing the reassignment.  So
> adding this
> > > > > > > state
> > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > MetadataResponse adds overhead for
> no real
> > > > > > > > > benefit.  In
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > common
> > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > where there is no ongoing
> reassignment, it
> > > > > > > would
> > > > > > > > > be 4
> > > > > > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > In general, I think we have a
> problem of
> > > > > > > > > oversharing in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we
> 10x or
> > > > > > > > > > > > > > > > > > > > > 100x
> > > > > > > the
> > > > > > > > > number
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > > > we support, we'll need to get
> stricter
> > > > > > > > > > > > > > > > > > > > > about
> > > > > > > giving
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > information they actually need,
> about the
> > > > > > > > > partitions
> > > > > > > > > > > they
> > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > > > > > about.  Reassignment state clearly
> falls
> > > > > > > > > > > > > > > > > > > > > in the
> > > > > > > > > > > category
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > state
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > isn't needed by clients (except
> very
> > > > > > > specialized
> > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Another important consideration
> here is
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > someone
> > > > > > > > > > > > > managing
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > ongoing reassignment wants the
> most
> > > > > > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > > > > > information,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > be found on the controller.
> Therefore
> > > > > > > > > > > > > > > > > > > > > adding
> > > > > > > this
> > > > > > > > > > > state
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > > > > > or describeTopics, which could
> contact any
> > > > > > > node in
> > > > > > > > > the
> > > > > > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Finally, adding this to listTopics
> or
> > > > > > > > > describeTopics
> > > > > > > > > > > > feels
> > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > > > > > API.  It's an extra boolean which
> > > > > > > > > > > > > > > > > > > > > interacts
> > > > > > > with
> > > > > > > > > other
> > > > > > > > > > > > > extra
> > > > > > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > > > > > like "show internal", etc. in
> weird ways.
> > > > > > > > > > > > > > > > > > > > > I
> > > > > > > think
> > > > > > > > > a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > API
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 3. As replicas come into sync,
> they
> > > > > > > > > > > > > > > > > > > > > > > will
> > > > > > > join
> > > > > > > > > the
> > > > > > > > > > > > ISR.
> > > > > > > > > > > > > > > Will we
> > > > > > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > > > > > target replicas joining the
> ISR before
> > > > > > > taking
> > > > > > > > > the
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > > > > > replicas set? Also, I assume
> that
> > > > > > > > > > > > > > > > > > > > > > > target
> > > > > > > > > replicas
> > > > > > > > > > > can
> > > > > > > > > > > > > > > still be
> > > > > > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > We'll take a replica out of the
> target
> > > > > > > replicas
> > > > > > > > > set
> > > > > > > > > > > as
> > > > > > > > > > > > > soon
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me
> clarify
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 4. Probably useful to mention
> > > > > > > > > > > > > > > > > > > > > > > permissions
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Good point.  I think
> > > > > > > alterPartitionAssignments
> > > > > > > > > should
> > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest
> permissions
> > > > > > > > > > > > > > > > > > > > > > will be
> > > > > > > > > > > > unchanged.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30
> AM Gwen
> > > > > > > Shapira <
> > > > > > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > > > > > Looks great, and will be
> awesome to
> > > > > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > this new
> > > > > > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23
> PM
> > > > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > McCabe
> > > > > > > > > <
> > > > > > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote
> for KIP-455:
> > > > > > > > > Create
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I
> think
> > > > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > KIP is
> > > > > > > > > > > > > important
> > > > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > > > > > many follow-on
> improvements to
> > > > > > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > reassignment
> > > > > > > > > > > > > (see
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > > > > > section, plus a lot of the
> other
> > > > > > > > > discussions
> > > > > > > > > > > > we've
> > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > > > > > reassignment).  It also
> furthers
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > important
> > > > > > > > > > > > > KIP-4
> > > > > > > > > > > > > > > goal
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > I made a few changes based
> on the
> > > > > > > > > discussion in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > > > > > Robert suggested, I
> removed the
> > > > > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > > for a partition before
> setting up
> > > > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > different
> > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > specific partition.  I
> also
> > > > > > > > > > > > > > > > > > > > > > > > > simplified
> > > > > > > the
> > > > > > > > > API
> > > > > > > > > > > a
> > > > > > > > > > > > > bit by
> > > > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > PartitionReassignment
> class which
> > > > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > used
> > > > > > > > > by
> > > > > > > > > > > both
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > alter
> > > > > > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > I modified the proposal so
> that we
> > > > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > > > > deprecate
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > old
> > > > > > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > > > > > rather than removing it
> completely.
> > > > > > > That
> > > > > > > > > > > should
> > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > > > > > tools some time to
> transition to
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > new
> > > > > > > > > API.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > To clarify a question
> Viktor
> > > > > > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > > > > > added a
> > > > > > > > > > > > note
> > > > > > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > > > > >
> kafka-reassign-partitions.sh will
> > > > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > use a
> > > > > > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product
> Manager |
> > > > > > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> Follow us:
> > > > > > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > > > > > https:
> > > > > > > > > > > >
> %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > > > > > 40microsoft.com
> %7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > > > > >
> f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > > > > >
> data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > > > > > served=0>
> > > > > > > > > > > > > |
> > > > > > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > > > > > <
> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnam06.safelinks.protection&amp;data=02%7C01%7Ckchitta%40microsoft.com%7C3e662dd1d60949f2ff6808d71b95e68f%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637008202957720437&amp;sdata=vdDj2GPhjVP5TQj1B4%2BoCjGU6cgDqGVYOEnqaMurz68%3D&amp;reserved=0
> .
> > > > > > > > > > > > > > > > > > > > > > > >
> outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > > > > > nfluent.io
> %2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com
> %7C092351102
> > > > > > > > > > > > > > > > > > > > > > > >
> 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > > > > >
> f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > > > > >
> 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > > > > >
> pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter |
> blog
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


-- 
Best,
Stanislav

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi all,

After some discussion with Jun and Stan, we decided that we should bump the version of the topics znode from 1 to 2.  The bump is backwards compatible (older brokers can read the v2 znode).  I have updated the KIP.

best,
Colin


On Thu, Aug 8, 2019, at 11:09, Colin McCabe wrote:
> Hi Koushik,
> 
> The vote for this KIP already passed.
> 
> See https://www.mail-archive.com/dev@kafka.apache.org/msg99636.html
> 
> best,
> Colin
> 
> On Thu, Aug 8, 2019, at 10:50, Koushik Chitta wrote:
> > Thanks Colin, George.   Can we restart the voting for this KIP.
> > 
> > Thanks,
> > Koushik 
> > 
> > -----Original Message-----
> > From: Colin McCabe <cm...@apache.org> 
> > Sent: Wednesday, August 7, 2019 5:17 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > Reassignment
> > 
> > On Wed, Aug 7, 2019, at 15:41, George Li wrote:
> > > This email seemed to get lost in the dev email server.  Resending. 
> > > 
> > > 
> > > On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li 
> > > <sq...@yahoo.com> wrote:
> > > 
> > > 
> > > The pending reassignments partitions would be reported as URP (Under 
> > > Replicated Partitions).  or maybe reported as a separate metrics of 
> > > RURP (Reassignment URP) since now we can derived from the new 
> > > AddingReplicas. An alert could be triggered based on this.
> > > 
> > 
> > Hi George,
> > 
> > I agree that this would be a great idea for follow up work.  Check out 
> > KIP-352, which discusses creating a such a metric. :)
> > 
> > > 
> > > 
> > > It would be nice if ListPartitionReassignmentResult could return the 
> > > "elapsed time/duration" of the current pending reassignments, the 
> > > calling client can flag those current long running reassignments and 
> > > alert.  However, what I would be interested is probably the total # of 
> > > pending reassignments because I will submit reassignments in batches, 
> > > e.g. 50 reassignments per batch.  If the pending reassignments # is 
> > > below that per batch #, submit more new reassignments = (per_batch_# - 
> > > pending_#).
> > > 
> > 
> > It is definitely useful to know what reassignments exist.  If you call 
> > ListPartitionReassignments, you can count how many results you get, in 
> > order to implement a policy like that.
> > 
> > I'm not sure if knowing how long reassignments have been in progress 
> > will be important or not.  I think we should give people some time to 
> > try out the new APIs and see what could be improved based on their 
> > experience.
> > 
> > > 
> > > 
> > > It seems currently, the ReplicaFetcher threads could quite easily crash 
> > > because of some exceptions. e.g. Java Out Of Memory, and would just 
> > > remain dead (jstack to dump threads to check the # of running 
> > > ReplicaFetcher threads) without getting restarted automatically, so 
> > > needs to bounce the broker.  It would be nice to make the 
> > > ReplicaFetcher more robust/resilient of catching more exceptions, and 
> > > if crashed, get restarted after some time. 
> > > 
> > 
> > This has definitely been an issue in the past, I agree.  Thankfully, we 
> > recently did improve the robustness of the ReplicaFetcher.  Check out 
> > "KIP-461: Improve Replica Fetcher behavior at handling partition 
> > failure."
> > 
> > cheers,
> > Colin
> > 
> > > 
> > > 
> > > Thanks,
> > > 
> > > George
> > > 
> > > 
> > > 
> > > On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote: 
> > > > Hi Koushik,
> > > > 
> > > > Thanks for the idea.  This KIP is already pretty big, so I think we'll have to consider ideas like this in follow-on KIPs.
> > > > 
> > > > In general, figuring out what's wrong with replication is a pretty tough problem.  If we had an API for this, we'd probably want it to be unified, and not specific to reassigning partitions.
> > > > 
> > > > regards,
> > > > Colin
> > > > 
> > > > 
> > > > On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > > > > Hey Colin,
> > > > > 
> > > > > Can the ListPartitionReassignmentsResult include the status of the 
> > > > > current reassignment progress of each partition? A reassignment can be 
> > > > > in progress for different reasons and the status can give the option to 
> > > > > alter the current reassignment.
> > > > > 
> > > > > Example -  A leaderISRRequest of a new assigned replicas can be 
> > > > > ignored/errored because of a storage exception.  And reassignment batch 
> > > > > will be waiting indefinitely for the new assigned replicas to be in 
> > > > > sync with the leader of the partition.  
> > > > > 	      Showing the status will give an option to alter the affected 
> > > > > partitions and allow the batch to complete reassignment.
> > > > > 
> > > > > OAR = {1, 2, 3} and RAR = {4,5,6}
> > > > > 
> > > > >      AR                 leader/isr
> > > > >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest 
> > > > > was lost/skipped for 5 and the reassignment operation will be waiting 
> > > > > indefinitely for the 5 to be insync.
> > > > > 
> > > > > 
> > > > > 
> > > > > Thanks,
> > > > > Koushik
> > > > > 
> > > > > -----Original Message-----
> > > > > From: Jun Rao <ju...@confluent.io> 
> > > > > Sent: Friday, August 2, 2019 10:04 AM
> > > > > To: dev <de...@kafka.apache.org>
> > > > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > > > > Reassignment
> > > > > 
> > > > > Hi, Colin,
> > > > > 
> > > > > First, since we are changing the format of LeaderAndIsrRequest, which 
> > > > > is an inter broker request, it seems that we will need IBP during 
> > > > > rolling upgrade. Could we add that to the compatibility section?
> > > > > 
> > > > > Regarding UnsupportedVersionException, even without ZK node version 
> > > > > bump, we probably want to only use the new ZK value fields after all 
> > > > > brokers have been upgraded to the new binary. Otherwise, the 
> > > > > reassignment task may not be completed if the controller changes to a 
> > > > > broker still on the old binary.
> > > > > IBP is one way to achieve that. The main thing is that we need some way 
> > > > > for the controller to deal with the new ZK fields. Dealing with the 
> > > > > additional ZK node version bump seems a small thing on top of that?
> > > > > 
> > > > > Thanks,
> > > > > 
> > > > > Jun
> > > > > 
> > > > > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:
> > > > > 
> > > > > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > > > > Hi, Colin,
> > > > > > >
> > > > > > > 10. Sounds good.
> > > > > > >
> > > > > > > 13. Our current convention is to bump up the version of ZK value if 
> > > > > > > there is any format change. For example, we have bumped up the 
> > > > > > > version of the value in /brokers/ids/nnn multiple times and all of 
> > > > > > > those changes are compatible (just adding new fields). This has the 
> > > > > > > slight benefit that it makes it clear there is a format change. 
> > > > > > > Rolling upgrades and downgrades can still be supported with the 
> > > > > > > version bump. For example, if you
> > > > > > downgrade
> > > > > > > from a compatible change, you can leave the new format in ZK and the 
> > > > > > > old code will only pick up fields relevant to the old version. 
> > > > > > > Upgrade will
> > > > > > be
> > > > > > > controlled by inter broker protocol.
> > > > > >
> > > > > > Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> > > > > > protocol version.  We also need to return UnsupportedVersionException 
> > > > > > from the alterPartitionReassignments and listPartitionReassignments 
> > > > > > APIs when the IBP is too low.  This sounds doable, although we might 
> > > > > > need a release note that upgrading the IBP is necessary to allow 
> > > > > > reassignment operations after an upgrade.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> > > > > > >
> > > > > > > > Hi Jun,
> > > > > > > >
> > > > > > > > Thanks for taking another look at this.
> > > > > > > >
> > > > > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > > > > Hi, Stan,
> > > > > > > > >
> > > > > > > > > Thanks for the explanation.
> > > > > > > > >
> > > > > > > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > > > > > > future
> > > > > > work,
> > > > > > > > > perhaps they should be added when we do the future work instead 
> > > > > > > > > of
> > > > > > now?
> > > > > > > >
> > > > > > > > I think this ties in with one of the big goals of this KIP, making 
> > > > > > > > it possible to distinguish reassigning replicas from normal replicas.
> > > > > > This is
> > > > > > > > the key to follow-on work like being able to ensure that 
> > > > > > > > partitions
> > > > > > with a
> > > > > > > > reassignment don't get falsely flagged as under-replicated in the
> > > > > > metrics,
> > > > > > > > or implementing reassignment quotas that don't accidentally affect
> > > > > > normal
> > > > > > > > replication traffic when a replica falls out of the ISR.
> > > > > > > >
> > > > > > > > For these follow-on improvements, we need to have that information 
> > > > > > > > in LeaderAndIsrRequest.  We could add the information in a 
> > > > > > > > follow-on KIP,
> > > > > > of
> > > > > > > > course, but then all the improvements are blocked on that 
> > > > > > > > follow-on
> > > > > > KIP.
> > > > > > > > That would slow things down for all of the downstream KIPs that 
> > > > > > > > are
> > > > > > blocked
> > > > > > > > on this.
> > > > > > > >
> > > > > > > > Also, to keep things consistent, I think it would be best if the
> > > > > > format of
> > > > > > > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > > > > > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > > > > > > I think
> > > > > > it
> > > > > > > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > > > > > > >
> > > > > > > > > > > Should we include those two fields in UpdateMetadata and
> > > > > > potentially
> > > > > > > > > > > Metadata requests too?
> > > > > > > >
> > > > > > > > We had some discussion earlier about how metadata responses to 
> > > > > > > > clients
> > > > > > are
> > > > > > > > getting too large, in part because they include a lot of 
> > > > > > > > information
> > > > > > that
> > > > > > > > most clients don't need (such as the ISR).  I think reassignment 
> > > > > > > > information definitely falls in the category of something a client
> > > > > > doesn't
> > > > > > > > need to know, so we shouldn't include it.
> > > > > > > >
> > > > > > > > A program like CruiseControl, or the command-line reassignment 
> > > > > > > > program, just wants to get the most up-to-date information about 
> > > > > > > > the state of reassigning partitions.  The MetadataRequest API 
> > > > > > > > wouldn't deliver that, because there are inherently delays in how 
> > > > > > > > we propagate metadata to brokers.  That's why the 
> > > > > > > > ListPartitionReassignments API is a better
> > > > > > choice
> > > > > > > > for those programs.  So I think if we added this information to 
> > > > > > > > the MetadataResponse, nobody would actually use it, and it would 
> > > > > > > > just use
> > > > > > up
> > > > > > > > more bandwidth.
> > > > > > > >
> > > > > > > > Of course, we can always revisit this later if we find a scenario
> > > > > > where a
> > > > > > > > producer or consumer would actually care about this.  But I think 
> > > > > > > > we
> > > > > > should
> > > > > > > > default to not adding stuff to the metadata response if we don't 
> > > > > > > > have a good use case in mind.
> > > > > > > >
> > > > > > > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > > > > > > we
> > > > > > > > cancel the
> > > > > > > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > > > > > > them
> > > > > > from
> > > > > > > > (the
> > > > > > > > > > > updated from the last-reassignment) R and TR, and starting
> > > > > > anew." In
> > > > > > > > this
> > > > > > > > > > > case, it seems that the controller needs to issue a 
> > > > > > > > > > > StopReplica
> > > > > > > > request to
> > > > > > > > > > > remove those unneeded replicas.
> > > > > > > >
> > > > > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > > > > >
> > > > > > > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > > > > > > AR
> > > > > > from
> > > > > > > > R,
> > > > > > > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> > > > > > cancel
> > > > > > > > the
> > > > > > > > > > > replica movements that have not yet completed." Similar to 
> > > > > > > > > > > the
> > > > > > > > above, it
> > > > > > > > > > > seems the controller needs to issue a StopReplica request to
> > > > > > remove
> > > > > > > > those
> > > > > > > > > > > unneeded replicas.
> > > > > > > >
> > > > > > > > Right.  Let's add this.
> > > > > > > >
> > > > > > > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> > > > > > should
> > > > > > > > we bump
> > > > > > > > > > > up the version number in the json value?
> > > > > > > >
> > > > > > > > The change to the zNode is backwards compatible, though.  Older 
> > > > > > > > brokers will continue to work, but just ignore the new fields.  If 
> > > > > > > > we bump that version number, then downgrades will require hand-editing zookeeper.
> > > > > > (Of
> > > > > > > > course downgrade isn't officially supported, but it would be nice 
> > > > > > > > not
> > > > > > to
> > > > > > > > break it if we don't need to...)  Changing the version number 
> > > > > > > > would
> > > > > > also
> > > > > > > > create problems during a rolling upgrade.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > > > > > > <cmccabe@apache.org
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi all,
> > > > > > > > > > > >
> > > > > > > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> > > > > > Robert
> > > > > > > > > > Barrett,
> > > > > > > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > > > > > > Jason
> > > > > > > > > > Gustafson,
> > > > > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > > > > sql_consulting@yahoo.com
> > > > > > > > > > > > .invalid>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > > > > George
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> > > > > > Shapira <
> > > > > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> > > > > > through
> > > > > > > > all
> > > > > > > > > > the
> > > > > > > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > > > > > > ended
> > > > > > up
> > > > > > > > with
> > > > > > > > > > a
> > > > > > > > > > > > > > great protocol.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > > > > > > jason@confluent.io>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > -Jason
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > > > > > > cmccabe@apache.org
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > > > > > > reflect
> > > > > > the
> > > > > > > > fact
> > > > > > > > > > > that
> > > > > > > > > > > > > > we've
> > > > > > > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > > > > > > for 3
> > > > > > days
> > > > > > > > as
> > > > > > > > > > > > usual.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > > > > > > Kozlovski
> > > > > > wrote:
> > > > > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > We have further iterated on the KIP in the
> > > > > > accompanying
> > > > > > > > > > > > discussion
> > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > > > > - we will internally use two collections to
> > > > > > represent a
> > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > > > > LeaderAndIsr has
> > > > > > > > > > been
> > > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > > > > > > collection,
> > > > > > > > > > > > but
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > > > > > > collections
> > > > > > - the
> > > > > > > > > > full
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > set,
> > > > > > > > > > > > > > > > > the replicas we are adding as part of this
> > > > > > reassignment
> > > > > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > > > > > > proper
> > > > > > > > rollback
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > > > > > > API
> > > > > > call
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > https://cwik
> > > > > > i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > > > > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > > > > C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> > > > > > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > > > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > > > > > > getting
> > > > > > a PR
> > > > > > > > > > done
> > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > > > > > > we'll
> > > > > > > > finish up
> > > > > > > > > > > the
> > > > > > > > > > > > > > vote.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > What is the estimated timeline to have this
> > > > > > > > implemented?
> > > > > > > > > > > If
> > > > > > > > > > > > any
> > > > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > > > > > > cancelling
> > > > > > > > > > > > reassignments,  I
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > > > > > > PDT,
> > > > > > Colin
> > > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> > > > > > individual
> > > > > > > > > > > > partitions to
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > > > > > > these
> > > > > > > > > > > operations
> > > > > > > > > > > > to
> > > > > > > > > > > > > > be at
> > > > > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > > > > > > good
> > > > > > > > > > > improvements
> > > > > > > > > > > > of
> > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > Cancellation
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > > > > > > when
> > > > > > the
> > > > > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > > > > > > empty
> > > > > > > > replicas
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > > > > > > implementing
> > > > > > the
> > > > > > > > the
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> > > > > > it
> > > > > > > > will
> > > > > > > > > > > make
> > > > > > > > > > > > > > KIP-236
> > > > > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > > > > > > all
> > > > > > > > pending
> > > > > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > > > > one just needs to do a
> > > > > > > > ListPartitionRessignmentRequest,
> > > > > > > > > > > > then
> > > > > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > > > > > > topic/partitions
> > > > > > in
> > > > > > > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > > > > > > PDT,
> > > > > > Colin
> > > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > > > > > > McCabe
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> > > > > > Gustafson
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > partition
> > > > > > > > > > > > level
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > > > > > > understand
> > > > > > > > > > that
> > > > > > > > > > > > > > there is
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > reassignment in progress. This makes
> > > > > > KIP-352 a
> > > > > > > > > > > trivial
> > > > > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > > > > > > whether
> > > > > > the
> > > > > > > > > > leader
> > > > > > > > > > > > and
> > > > > > > > > > > > > > isr
> > > > > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > > > right place to store the target
> > > > > > reassignment.
> > > > > > > > It
> > > > > > > > > > is a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > > odd
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > > > > > > from
> > > > > > the
> > > > > > > > > > > current
> > > > > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > > > > assume the thinking is probably that
> > > > > > although
> > > > > > > > the
> > > > > > > > > > > > current
> > > > > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > > > > > > znode as
> > > > > > > > well, it
> > > > > > > > > > > is
> > > > > > > > > > > > > > hard to
> > > > > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> > > > > > But
> > > > > > > > if we
> > > > > > > > > > > > have no
> > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > > > > the assignment znode, do you see a
> > > > > > downside to
> > > > > > > > > > > storing
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > > > > > > better
> > > > > > to
> > > > > > > > keep
> > > > > > > > > > the
> > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > > > > > > current
> > > > > > > > > > assignment,
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > > > > > > we
> > > > > > should
> > > > > > > > > > stick
> > > > > > > > > > > > with
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > > > > > > reassignment
> > > > > > state
> > > > > > > > into
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > > > > The
> > > > > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > > > > > > epoch
> > > > > > for the
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > epoch
> > > > > > > > > > > > resides
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > znode anyway.  I agree there is some
> > > > > > inconsistency
> > > > > > > > > > here,
> > > > > > > > > > > > but
> > > > > > > > > > > > > > so be
> > > > > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > > > > > > data
> > > > > > > > > > structures,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > might
> > > > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > > > > > > will
> > > > > > work
> > > > > > > > fine
> > > > > > > > > > > and
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > > > > > > altering,
> > > > > > > > right?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > > > > > > reads
> > > > > > a
> > > > > > > > little
> > > > > > > > > > > > > > better, and
> > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > > > > > > +the
> > > > > > wiki.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > API? In
> > > > > > > > > > other
> > > > > > > > > > > > > > words,
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> > > > > > probably
> > > > > > > > > > need
> > > > > > > > > > > a
> > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> > > > > > alternative I
> > > > > > > > can
> > > > > > > > > > > > think of
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > > > request
> > > > > > > > > > as
> > > > > > > > > > > > the
> > > > > > > > > > > > > > union
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > > > > > > can't
> > > > > > think
> > > > > > > > of
> > > > > > > > > > any
> > > > > > > > > > > > > > benefit
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > > > > > > this, we
> > > > > > > > > > probably
> > > > > > > > > > > > > > wouldn't
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > > > > > > think on
> > > > > > > > balance,
> > > > > > > > > > > > you're
> > > > > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > > > > > > with
> > > > > > the
> > > > > > > > > > replica
> > > > > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > > > > > > information is
> > > > > > > > > > > > > > available in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > > > > > > this,
> > > > > > we'll
> > > > > > > > need
> > > > > > > > > > a
> > > > > > > > > > > > flag
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > > > > > > show me
> > > > > > > > > > > reassigning
> > > > > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > > > > > > stick
> > > > > > with
> > > > > > > > the
> > > > > > > > > > > > original
> > > > > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > > > > ListPartitionReassignments
> > > > > > > > > > > > API.
> > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > > > > > > or
> > > > > > > > Consumer
> > > > > > > > > > > needs
> > > > > > > > > > > > to
> > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > about a reassignment.  They should just be
> > > > > > notified
> > > > > > > > > > when
> > > > > > > > > > > > the
> > > > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > > > partitions changes, which doesn't require
> > > > > > changes
> > > > > > > > to
> > > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > > > > > > client
> > > > > > only
> > > > > > > > cares
> > > > > > > > > > if
> > > > > > > > > > > > > > someone
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > managing the reassignment.  So adding this
> > > > > > state
> > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > > > > > > benefit.  In
> > > > > > > > > > > the
> > > > > > > > > > > > > > common
> > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> > > > > > would
> > > > > > > > be 4
> > > > > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > In general, I think we have a problem of
> > > > > > > > oversharing in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > > > > > > 100x
> > > > > > the
> > > > > > > > number
> > > > > > > > > > > of
> > > > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > > > > > > about
> > > > > > giving
> > > > > > > > > > > clients
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > information they actually need, about the
> > > > > > > > partitions
> > > > > > > > > > they
> > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > > > > > > in the
> > > > > > > > > > category
> > > > > > > > > > > of
> > > > > > > > > > > > > > state
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > isn't needed by clients (except very
> > > > > > specialized
> > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > someone
> > > > > > > > > > > > managing
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > > > > information,
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > > > > > > adding
> > > > > > this
> > > > > > > > > > state
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > > > > or describeTopics, which could contact any
> > > > > > node in
> > > > > > > > the
> > > > > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > > > > > > describeTopics
> > > > > > > > > > > feels
> > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > > > > > > interacts
> > > > > > with
> > > > > > > > other
> > > > > > > > > > > > extra
> > > > > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > > > > > > I
> > > > > > think
> > > > > > > > a
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > API
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > > > > > > will
> > > > > > join
> > > > > > > > the
> > > > > > > > > > > ISR.
> > > > > > > > > > > > > > Will we
> > > > > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > > > > target replicas joining the ISR before
> > > > > > taking
> > > > > > > > the
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > > > > > > target
> > > > > > > > replicas
> > > > > > > > > > can
> > > > > > > > > > > > > > still be
> > > > > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > We'll take a replica out of the target
> > > > > > replicas
> > > > > > > > set
> > > > > > > > > > as
> > > > > > > > > > > > soon
> > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > in
> > > > > > > > the
> > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > > > > > > permissions
> > > > > > for
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Good point.  I think
> > > > > > alterPartitionAssignments
> > > > > > > > should
> > > > > > > > > > > > require
> > > > > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > > > > > > will be
> > > > > > > > > > > unchanged.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> > > > > > Shapira <
> > > > > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > this new
> > > > > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > > McCabe
> > > > > > > > <
> > > > > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > > > > > > Create
> > > > > > > > > > an
> > > > > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > KIP is
> > > > > > > > > > > > important
> > > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > > > > reassignment
> > > > > > > > > > > > (see
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > > > > > > discussions
> > > > > > > > > > > we've
> > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > important
> > > > > > > > > > > > KIP-4
> > > > > > > > > > > > > > goal
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > > > > > > discussion in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > explicitly
> > > > > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > different
> > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > > > > > > simplified
> > > > > > the
> > > > > > > > API
> > > > > > > > > > a
> > > > > > > > > > > > bit by
> > > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > used
> > > > > > > > by
> > > > > > > > > > both
> > > > > > > > > > > > the
> > > > > > > > > > > > > > alter
> > > > > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > > > deprecate
> > > > > > > > > > > > the
> > > > > > > > > > > > > > old
> > > > > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > > > > rather than removing it completely.
> > > > > > That
> > > > > > > > > > should
> > > > > > > > > > > > give
> > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > new
> > > > > > > > API.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > > > > added a
> > > > > > > > > > > note
> > > > > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > use a
> > > > > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > > > > https:
> > > > > > > > > > > %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > > > > served=0>
> > > > > > > > > > > > |
> > > > > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnam06.safelinks.protection&amp;data=02%7C01%7Ckchitta%40microsoft.com%7C3e662dd1d60949f2ff6808d71b95e68f%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637008202957720437&amp;sdata=vdDj2GPhjVP5TQj1B4%2BoCjGU6cgDqGVYOEnqaMurz68%3D&amp;reserved=0.
> > > > > > > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > > > > 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > --
> > > > > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > 
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi Koushik,

The vote for this KIP already passed.

See https://www.mail-archive.com/dev@kafka.apache.org/msg99636.html

best,
Colin

On Thu, Aug 8, 2019, at 10:50, Koushik Chitta wrote:
> Thanks Colin, George.   Can we restart the voting for this KIP.
> 
> Thanks,
> Koushik 
> 
> -----Original Message-----
> From: Colin McCabe <cm...@apache.org> 
> Sent: Wednesday, August 7, 2019 5:17 PM
> To: dev@kafka.apache.org
> Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> Reassignment
> 
> On Wed, Aug 7, 2019, at 15:41, George Li wrote:
> > This email seemed to get lost in the dev email server.  Resending. 
> > 
> > 
> > On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li 
> > <sq...@yahoo.com> wrote:
> > 
> > 
> > The pending reassignments partitions would be reported as URP (Under 
> > Replicated Partitions).  or maybe reported as a separate metrics of 
> > RURP (Reassignment URP) since now we can derived from the new 
> > AddingReplicas. An alert could be triggered based on this.
> > 
> 
> Hi George,
> 
> I agree that this would be a great idea for follow up work.  Check out 
> KIP-352, which discusses creating a such a metric. :)
> 
> > 
> > 
> > It would be nice if ListPartitionReassignmentResult could return the 
> > "elapsed time/duration" of the current pending reassignments, the 
> > calling client can flag those current long running reassignments and 
> > alert.  However, what I would be interested is probably the total # of 
> > pending reassignments because I will submit reassignments in batches, 
> > e.g. 50 reassignments per batch.  If the pending reassignments # is 
> > below that per batch #, submit more new reassignments = (per_batch_# - 
> > pending_#).
> > 
> 
> It is definitely useful to know what reassignments exist.  If you call 
> ListPartitionReassignments, you can count how many results you get, in 
> order to implement a policy like that.
> 
> I'm not sure if knowing how long reassignments have been in progress 
> will be important or not.  I think we should give people some time to 
> try out the new APIs and see what could be improved based on their 
> experience.
> 
> > 
> > 
> > It seems currently, the ReplicaFetcher threads could quite easily crash 
> > because of some exceptions. e.g. Java Out Of Memory, and would just 
> > remain dead (jstack to dump threads to check the # of running 
> > ReplicaFetcher threads) without getting restarted automatically, so 
> > needs to bounce the broker.  It would be nice to make the 
> > ReplicaFetcher more robust/resilient of catching more exceptions, and 
> > if crashed, get restarted after some time. 
> > 
> 
> This has definitely been an issue in the past, I agree.  Thankfully, we 
> recently did improve the robustness of the ReplicaFetcher.  Check out 
> "KIP-461: Improve Replica Fetcher behavior at handling partition 
> failure."
> 
> cheers,
> Colin
> 
> > 
> > 
> > Thanks,
> > 
> > George
> > 
> > 
> > 
> > On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote: 
> > > Hi Koushik,
> > > 
> > > Thanks for the idea.  This KIP is already pretty big, so I think we'll have to consider ideas like this in follow-on KIPs.
> > > 
> > > In general, figuring out what's wrong with replication is a pretty tough problem.  If we had an API for this, we'd probably want it to be unified, and not specific to reassigning partitions.
> > > 
> > > regards,
> > > Colin
> > > 
> > > 
> > > On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > > > Hey Colin,
> > > > 
> > > > Can the ListPartitionReassignmentsResult include the status of the 
> > > > current reassignment progress of each partition? A reassignment can be 
> > > > in progress for different reasons and the status can give the option to 
> > > > alter the current reassignment.
> > > > 
> > > > Example -  A leaderISRRequest of a new assigned replicas can be 
> > > > ignored/errored because of a storage exception.  And reassignment batch 
> > > > will be waiting indefinitely for the new assigned replicas to be in 
> > > > sync with the leader of the partition.  
> > > > 	      Showing the status will give an option to alter the affected 
> > > > partitions and allow the batch to complete reassignment.
> > > > 
> > > > OAR = {1, 2, 3} and RAR = {4,5,6}
> > > > 
> > > >      AR                 leader/isr
> > > >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest 
> > > > was lost/skipped for 5 and the reassignment operation will be waiting 
> > > > indefinitely for the 5 to be insync.
> > > > 
> > > > 
> > > > 
> > > > Thanks,
> > > > Koushik
> > > > 
> > > > -----Original Message-----
> > > > From: Jun Rao <ju...@confluent.io> 
> > > > Sent: Friday, August 2, 2019 10:04 AM
> > > > To: dev <de...@kafka.apache.org>
> > > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > > > Reassignment
> > > > 
> > > > Hi, Colin,
> > > > 
> > > > First, since we are changing the format of LeaderAndIsrRequest, which 
> > > > is an inter broker request, it seems that we will need IBP during 
> > > > rolling upgrade. Could we add that to the compatibility section?
> > > > 
> > > > Regarding UnsupportedVersionException, even without ZK node version 
> > > > bump, we probably want to only use the new ZK value fields after all 
> > > > brokers have been upgraded to the new binary. Otherwise, the 
> > > > reassignment task may not be completed if the controller changes to a 
> > > > broker still on the old binary.
> > > > IBP is one way to achieve that. The main thing is that we need some way 
> > > > for the controller to deal with the new ZK fields. Dealing with the 
> > > > additional ZK node version bump seems a small thing on top of that?
> > > > 
> > > > Thanks,
> > > > 
> > > > Jun
> > > > 
> > > > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:
> > > > 
> > > > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > > > Hi, Colin,
> > > > > >
> > > > > > 10. Sounds good.
> > > > > >
> > > > > > 13. Our current convention is to bump up the version of ZK value if 
> > > > > > there is any format change. For example, we have bumped up the 
> > > > > > version of the value in /brokers/ids/nnn multiple times and all of 
> > > > > > those changes are compatible (just adding new fields). This has the 
> > > > > > slight benefit that it makes it clear there is a format change. 
> > > > > > Rolling upgrades and downgrades can still be supported with the 
> > > > > > version bump. For example, if you
> > > > > downgrade
> > > > > > from a compatible change, you can leave the new format in ZK and the 
> > > > > > old code will only pick up fields relevant to the old version. 
> > > > > > Upgrade will
> > > > > be
> > > > > > controlled by inter broker protocol.
> > > > >
> > > > > Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> > > > > protocol version.  We also need to return UnsupportedVersionException 
> > > > > from the alterPartitionReassignments and listPartitionReassignments 
> > > > > APIs when the IBP is too low.  This sounds doable, although we might 
> > > > > need a release note that upgrading the IBP is necessary to allow 
> > > > > reassignment operations after an upgrade.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > >
> > > > > > > Thanks for taking another look at this.
> > > > > > >
> > > > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > > > Hi, Stan,
> > > > > > > >
> > > > > > > > Thanks for the explanation.
> > > > > > > >
> > > > > > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > > > > > future
> > > > > work,
> > > > > > > > perhaps they should be added when we do the future work instead 
> > > > > > > > of
> > > > > now?
> > > > > > >
> > > > > > > I think this ties in with one of the big goals of this KIP, making 
> > > > > > > it possible to distinguish reassigning replicas from normal replicas.
> > > > > This is
> > > > > > > the key to follow-on work like being able to ensure that 
> > > > > > > partitions
> > > > > with a
> > > > > > > reassignment don't get falsely flagged as under-replicated in the
> > > > > metrics,
> > > > > > > or implementing reassignment quotas that don't accidentally affect
> > > > > normal
> > > > > > > replication traffic when a replica falls out of the ISR.
> > > > > > >
> > > > > > > For these follow-on improvements, we need to have that information 
> > > > > > > in LeaderAndIsrRequest.  We could add the information in a 
> > > > > > > follow-on KIP,
> > > > > of
> > > > > > > course, but then all the improvements are blocked on that 
> > > > > > > follow-on
> > > > > KIP.
> > > > > > > That would slow things down for all of the downstream KIPs that 
> > > > > > > are
> > > > > blocked
> > > > > > > on this.
> > > > > > >
> > > > > > > Also, to keep things consistent, I think it would be best if the
> > > > > format of
> > > > > > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > > > > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > > > > > I think
> > > > > it
> > > > > > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > > > > > >
> > > > > > > > > > Should we include those two fields in UpdateMetadata and
> > > > > potentially
> > > > > > > > > > Metadata requests too?
> > > > > > >
> > > > > > > We had some discussion earlier about how metadata responses to 
> > > > > > > clients
> > > > > are
> > > > > > > getting too large, in part because they include a lot of 
> > > > > > > information
> > > > > that
> > > > > > > most clients don't need (such as the ISR).  I think reassignment 
> > > > > > > information definitely falls in the category of something a client
> > > > > doesn't
> > > > > > > need to know, so we shouldn't include it.
> > > > > > >
> > > > > > > A program like CruiseControl, or the command-line reassignment 
> > > > > > > program, just wants to get the most up-to-date information about 
> > > > > > > the state of reassigning partitions.  The MetadataRequest API 
> > > > > > > wouldn't deliver that, because there are inherently delays in how 
> > > > > > > we propagate metadata to brokers.  That's why the 
> > > > > > > ListPartitionReassignments API is a better
> > > > > choice
> > > > > > > for those programs.  So I think if we added this information to 
> > > > > > > the MetadataResponse, nobody would actually use it, and it would 
> > > > > > > just use
> > > > > up
> > > > > > > more bandwidth.
> > > > > > >
> > > > > > > Of course, we can always revisit this later if we find a scenario
> > > > > where a
> > > > > > > producer or consumer would actually care about this.  But I think 
> > > > > > > we
> > > > > should
> > > > > > > default to not adding stuff to the metadata response if we don't 
> > > > > > > have a good use case in mind.
> > > > > > >
> > > > > > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > > > > > we
> > > > > > > cancel the
> > > > > > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > > > > > them
> > > > > from
> > > > > > > (the
> > > > > > > > > > updated from the last-reassignment) R and TR, and starting
> > > > > anew." In
> > > > > > > this
> > > > > > > > > > case, it seems that the controller needs to issue a 
> > > > > > > > > > StopReplica
> > > > > > > request to
> > > > > > > > > > remove those unneeded replicas.
> > > > > > >
> > > > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > > > >
> > > > > > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > > > > > AR
> > > > > from
> > > > > > > R,
> > > > > > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> > > > > cancel
> > > > > > > the
> > > > > > > > > > replica movements that have not yet completed." Similar to 
> > > > > > > > > > the
> > > > > > > above, it
> > > > > > > > > > seems the controller needs to issue a StopReplica request to
> > > > > remove
> > > > > > > those
> > > > > > > > > > unneeded replicas.
> > > > > > >
> > > > > > > Right.  Let's add this.
> > > > > > >
> > > > > > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> > > > > should
> > > > > > > we bump
> > > > > > > > > > up the version number in the json value?
> > > > > > >
> > > > > > > The change to the zNode is backwards compatible, though.  Older 
> > > > > > > brokers will continue to work, but just ignore the new fields.  If 
> > > > > > > we bump that version number, then downgrades will require hand-editing zookeeper.
> > > > > (Of
> > > > > > > course downgrade isn't officially supported, but it would be nice 
> > > > > > > not
> > > > > to
> > > > > > > break it if we don't need to...)  Changing the version number 
> > > > > > > would
> > > > > also
> > > > > > > create problems during a rolling upgrade.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > > > > > <cmccabe@apache.org
> > > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi all,
> > > > > > > > > > >
> > > > > > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> > > > > Robert
> > > > > > > > > Barrett,
> > > > > > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > > > > > Jason
> > > > > > > > > Gustafson,
> > > > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > > > sql_consulting@yahoo.com
> > > > > > > > > > > .invalid>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > > > George
> > > > > > > > > > > > >
> > > > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> > > > > Shapira <
> > > > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> > > > > through
> > > > > > > all
> > > > > > > > > the
> > > > > > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > > > > > ended
> > > > > up
> > > > > > > with
> > > > > > > > > a
> > > > > > > > > > > > > great protocol.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > > > > > jason@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > -Jason
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > > > > > cmccabe@apache.org
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > > > > > reflect
> > > > > the
> > > > > > > fact
> > > > > > > > > > that
> > > > > > > > > > > > > we've
> > > > > > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > > > > > for 3
> > > > > days
> > > > > > > as
> > > > > > > > > > > usual.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > > > > > Kozlovski
> > > > > wrote:
> > > > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > We have further iterated on the KIP in the
> > > > > accompanying
> > > > > > > > > > > discussion
> > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > > > - we will internally use two collections to
> > > > > represent a
> > > > > > > > > > > reassignment
> > > > > > > > > > > > > -
> > > > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > > > LeaderAndIsr has
> > > > > > > > > been
> > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > > > > > collection,
> > > > > > > > > > > but
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > > > > > collections
> > > > > - the
> > > > > > > > > full
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > set,
> > > > > > > > > > > > > > > > the replicas we are adding as part of this
> > > > > reassignment
> > > > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > > > > > proper
> > > > > > > rollback
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > > > > > API
> > > > > call
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > https://cwik
> > > > > i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > > > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > > > C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> > > > > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > > > > > getting
> > > > > a PR
> > > > > > > > > done
> > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > > > > > we'll
> > > > > > > finish up
> > > > > > > > > > the
> > > > > > > > > > > > > vote.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > What is the estimated timeline to have this
> > > > > > > implemented?
> > > > > > > > > > If
> > > > > > > > > > > any
> > > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > > > > > cancelling
> > > > > > > > > > > reassignments,  I
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > > > > > PDT,
> > > > > Colin
> > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> > > > > individual
> > > > > > > > > > > partitions to
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > > > > > these
> > > > > > > > > > operations
> > > > > > > > > > > to
> > > > > > > > > > > > > be at
> > > > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > > > > > good
> > > > > > > > > > improvements
> > > > > > > > > > > of
> > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > Cancellation
> > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > > > > > when
> > > > > the
> > > > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > > > > > empty
> > > > > > > replicas
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > > > > > implementing
> > > > > the
> > > > > > > the
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> > > > > it
> > > > > > > will
> > > > > > > > > > make
> > > > > > > > > > > > > KIP-236
> > > > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > > > > > all
> > > > > > > pending
> > > > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > > > one just needs to do a
> > > > > > > ListPartitionRessignmentRequest,
> > > > > > > > > > > then
> > > > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > > > > > topic/partitions
> > > > > in
> > > > > > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > > > > > PDT,
> > > > > Colin
> > > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > > > > > McCabe
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> > > > > Gustafson
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > partition
> > > > > > > > > > > level
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > > > > > understand
> > > > > > > > > that
> > > > > > > > > > > > > there is
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > reassignment in progress. This makes
> > > > > KIP-352 a
> > > > > > > > > > trivial
> > > > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > > > > > whether
> > > > > the
> > > > > > > > > leader
> > > > > > > > > > > and
> > > > > > > > > > > > > isr
> > > > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > > right place to store the target
> > > > > reassignment.
> > > > > > > It
> > > > > > > > > is a
> > > > > > > > > > > bit
> > > > > > > > > > > > > odd
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > > > > > from
> > > > > the
> > > > > > > > > > current
> > > > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > > > assume the thinking is probably that
> > > > > although
> > > > > > > the
> > > > > > > > > > > current
> > > > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > > > > > znode as
> > > > > > > well, it
> > > > > > > > > > is
> > > > > > > > > > > > > hard to
> > > > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> > > > > But
> > > > > > > if we
> > > > > > > > > > > have no
> > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > > > the assignment znode, do you see a
> > > > > downside to
> > > > > > > > > > storing
> > > > > > > > > > > the
> > > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > > > > > better
> > > > > to
> > > > > > > keep
> > > > > > > > > the
> > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > > > > > current
> > > > > > > > > assignment,
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > > > > > we
> > > > > should
> > > > > > > > > stick
> > > > > > > > > > > with
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > > > > > reassignment
> > > > > state
> > > > > > > into
> > > > > > > > > > > > > > > > > > >
> > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > > > The
> > > > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > > > > > epoch
> > > > > for the
> > > > > > > > > > > partition
> > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > > > > > leader
> > > > > > > epoch
> > > > > > > > > > > resides
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > znode anyway.  I agree there is some
> > > > > inconsistency
> > > > > > > > > here,
> > > > > > > > > > > but
> > > > > > > > > > > > > so be
> > > > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > > > > > data
> > > > > > > > > structures,
> > > > > > > > > > > we
> > > > > > > > > > > > > might
> > > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > > > > > will
> > > > > work
> > > > > > > fine
> > > > > > > > > > and
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > > > > > altering,
> > > > > > > right?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > > > > > reads
> > > > > a
> > > > > > > little
> > > > > > > > > > > > > better, and
> > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > > > > > +the
> > > > > wiki.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > API? In
> > > > > > > > > other
> > > > > > > > > > > > > words,
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> > > > > probably
> > > > > > > > > need
> > > > > > > > > > a
> > > > > > > > > > > > > change
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> > > > > alternative I
> > > > > > > can
> > > > > > > > > > > think of
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > > request
> > > > > > > > > as
> > > > > > > > > > > the
> > > > > > > > > > > > > union
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > > > > > can't
> > > > > think
> > > > > > > of
> > > > > > > > > any
> > > > > > > > > > > > > benefit
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > > > > > this, we
> > > > > > > > > probably
> > > > > > > > > > > > > wouldn't
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > > > > > think on
> > > > > > > balance,
> > > > > > > > > > > you're
> > > > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > > > > > with
> > > > > the
> > > > > > > > > replica
> > > > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > > > > > information is
> > > > > > > > > > > > > available in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > > > > > this,
> > > > > we'll
> > > > > > > need
> > > > > > > > > a
> > > > > > > > > > > flag
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > > > > > show me
> > > > > > > > > > reassigning
> > > > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > > > > > stick
> > > > > with
> > > > > > > the
> > > > > > > > > > > original
> > > > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > > > ListPartitionReassignments
> > > > > > > > > > > API.
> > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > > > > > or
> > > > > > > Consumer
> > > > > > > > > > needs
> > > > > > > > > > > to
> > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > about a reassignment.  They should just be
> > > > > notified
> > > > > > > > > when
> > > > > > > > > > > the
> > > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > > partitions changes, which doesn't require
> > > > > changes
> > > > > > > to
> > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > > > > > client
> > > > > only
> > > > > > > cares
> > > > > > > > > if
> > > > > > > > > > > > > someone
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > managing the reassignment.  So adding this
> > > > > state
> > > > > > > to the
> > > > > > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > > > > > benefit.  In
> > > > > > > > > > the
> > > > > > > > > > > > > common
> > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> > > > > would
> > > > > > > be 4
> > > > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > In general, I think we have a problem of
> > > > > > > oversharing in
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > > > > > 100x
> > > > > the
> > > > > > > number
> > > > > > > > > > of
> > > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > > > > > about
> > > > > giving
> > > > > > > > > > clients
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > information they actually need, about the
> > > > > > > partitions
> > > > > > > > > they
> > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > > > > > in the
> > > > > > > > > category
> > > > > > > > > > of
> > > > > > > > > > > > > state
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > isn't needed by clients (except very
> > > > > specialized
> > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > > > > > that
> > > > > > > someone
> > > > > > > > > > > managing
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > > > information,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > > > > > adding
> > > > > this
> > > > > > > > > state
> > > > > > > > > > to
> > > > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > > > or describeTopics, which could contact any
> > > > > node in
> > > > > > > the
> > > > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > > > > > describeTopics
> > > > > > > > > > feels
> > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > > > > > interacts
> > > > > with
> > > > > > > other
> > > > > > > > > > > extra
> > > > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > > > > > I
> > > > > think
> > > > > > > a
> > > > > > > > > > > separate
> > > > > > > > > > > > > API
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > > > > > will
> > > > > join
> > > > > > > the
> > > > > > > > > > ISR.
> > > > > > > > > > > > > Will we
> > > > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > > > target replicas joining the ISR before
> > > > > taking
> > > > > > > the
> > > > > > > > > > > replica
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > > > > > target
> > > > > > > replicas
> > > > > > > > > can
> > > > > > > > > > > > > still be
> > > > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > We'll take a replica out of the target
> > > > > replicas
> > > > > > > set
> > > > > > > > > as
> > > > > > > > > > > soon
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > > > > > this
> > > > > in
> > > > > > > the
> > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > > > > > permissions
> > > > > for
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Good point.  I think
> > > > > alterPartitionAssignments
> > > > > > > should
> > > > > > > > > > > require
> > > > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > > > > > will be
> > > > > > > > > > unchanged.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> > > > > Shapira <
> > > > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > > > > > have
> > > > > > > this new
> > > > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > > > > > Colin
> > > > > McCabe
> > > > > > > <
> > > > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > > > > > Create
> > > > > > > > > an
> > > > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > KIP is
> > > > > > > > > > > important
> > > > > > > > > > > > > since
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > > > reassignment
> > > > > > > > > > > (see
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > > > > > discussions
> > > > > > > > > > we've
> > > > > > > > > > > had
> > > > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > important
> > > > > > > > > > > KIP-4
> > > > > > > > > > > > > goal
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > > > > > discussion in
> > > > > > > > > > the
> > > > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > explicitly
> > > > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > different
> > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > > > > > simplified
> > > > > the
> > > > > > > API
> > > > > > > > > a
> > > > > > > > > > > bit by
> > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > > > > > is
> > > > > used
> > > > > > > by
> > > > > > > > > both
> > > > > > > > > > > the
> > > > > > > > > > > > > alter
> > > > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > > deprecate
> > > > > > > > > > > the
> > > > > > > > > > > > > old
> > > > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > > > rather than removing it completely.
> > > > > That
> > > > > > > > > should
> > > > > > > > > > > give
> > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > new
> > > > > > > API.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > > > added a
> > > > > > > > > > note
> > > > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > > > > > now
> > > > > use a
> > > > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > > > https:
> > > > > > > > > > %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > > > served=0>
> > > > > > > > > > > |
> > > > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnam06.safelinks.protection&amp;data=02%7C01%7Ckchitta%40microsoft.com%7C3e662dd1d60949f2ff6808d71b95e68f%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637008202957720437&amp;sdata=vdDj2GPhjVP5TQj1B4%2BoCjGU6cgDqGVYOEnqaMurz68%3D&amp;reserved=0.
> > > > > > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > > > 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > --
> > > > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > 
> >
>

RE: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Koushik Chitta <kc...@microsoft.com>.
Thanks Colin, George.   Can we restart the voting for this KIP.

Thanks,
Koushik 

-----Original Message-----
From: Colin McCabe <cm...@apache.org> 
Sent: Wednesday, August 7, 2019 5:17 PM
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

On Wed, Aug 7, 2019, at 15:41, George Li wrote:
> This email seemed to get lost in the dev email server.  Resending. 
> 
> 
> On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li 
> <sq...@yahoo.com> wrote:
> 
> 
> The pending reassignments partitions would be reported as URP (Under 
> Replicated Partitions).  or maybe reported as a separate metrics of 
> RURP (Reassignment URP) since now we can derived from the new 
> AddingReplicas. An alert could be triggered based on this.
> 

Hi George,

I agree that this would be a great idea for follow up work.  Check out KIP-352, which discusses creating a such a metric. :)

> 
> 
> It would be nice if ListPartitionReassignmentResult could return the 
> "elapsed time/duration" of the current pending reassignments, the 
> calling client can flag those current long running reassignments and 
> alert.  However, what I would be interested is probably the total # of 
> pending reassignments because I will submit reassignments in batches, 
> e.g. 50 reassignments per batch.  If the pending reassignments # is 
> below that per batch #, submit more new reassignments = (per_batch_# - 
> pending_#).
> 

It is definitely useful to know what reassignments exist.  If you call ListPartitionReassignments, you can count how many results you get, in order to implement a policy like that.

I'm not sure if knowing how long reassignments have been in progress will be important or not.  I think we should give people some time to try out the new APIs and see what could be improved based on their experience.

> 
> 
> It seems currently, the ReplicaFetcher threads could quite easily crash 
> because of some exceptions. e.g. Java Out Of Memory, and would just 
> remain dead (jstack to dump threads to check the # of running 
> ReplicaFetcher threads) without getting restarted automatically, so 
> needs to bounce the broker.  It would be nice to make the 
> ReplicaFetcher more robust/resilient of catching more exceptions, and 
> if crashed, get restarted after some time. 
> 

This has definitely been an issue in the past, I agree.  Thankfully, we recently did improve the robustness of the ReplicaFetcher.  Check out "KIP-461: Improve Replica Fetcher behavior at handling partition failure."

cheers,
Colin

> 
> 
> Thanks,
> 
> George
> 
> 
> 
> On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote: 
> > Hi Koushik,
> > 
> > Thanks for the idea.  This KIP is already pretty big, so I think we'll have to consider ideas like this in follow-on KIPs.
> > 
> > In general, figuring out what's wrong with replication is a pretty tough problem.  If we had an API for this, we'd probably want it to be unified, and not specific to reassigning partitions.
> > 
> > regards,
> > Colin
> > 
> > 
> > On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > > Hey Colin,
> > > 
> > > Can the ListPartitionReassignmentsResult include the status of the 
> > > current reassignment progress of each partition? A reassignment can be 
> > > in progress for different reasons and the status can give the option to 
> > > alter the current reassignment.
> > > 
> > > Example -  A leaderISRRequest of a new assigned replicas can be 
> > > ignored/errored because of a storage exception.  And reassignment batch 
> > > will be waiting indefinitely for the new assigned replicas to be in 
> > > sync with the leader of the partition.  
> > > 	      Showing the status will give an option to alter the affected 
> > > partitions and allow the batch to complete reassignment.
> > > 
> > > OAR = {1, 2, 3} and RAR = {4,5,6}
> > > 
> > >      AR                 leader/isr
> > >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest 
> > > was lost/skipped for 5 and the reassignment operation will be waiting 
> > > indefinitely for the 5 to be insync.
> > > 
> > > 
> > > 
> > > Thanks,
> > > Koushik
> > > 
> > > -----Original Message-----
> > > From: Jun Rao <ju...@confluent.io> 
> > > Sent: Friday, August 2, 2019 10:04 AM
> > > To: dev <de...@kafka.apache.org>
> > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > > Reassignment
> > > 
> > > Hi, Colin,
> > > 
> > > First, since we are changing the format of LeaderAndIsrRequest, which 
> > > is an inter broker request, it seems that we will need IBP during 
> > > rolling upgrade. Could we add that to the compatibility section?
> > > 
> > > Regarding UnsupportedVersionException, even without ZK node version 
> > > bump, we probably want to only use the new ZK value fields after all 
> > > brokers have been upgraded to the new binary. Otherwise, the 
> > > reassignment task may not be completed if the controller changes to a 
> > > broker still on the old binary.
> > > IBP is one way to achieve that. The main thing is that we need some way 
> > > for the controller to deal with the new ZK fields. Dealing with the 
> > > additional ZK node version bump seems a small thing on top of that?
> > > 
> > > Thanks,
> > > 
> > > Jun
> > > 
> > > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:
> > > 
> > > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > > Hi, Colin,
> > > > >
> > > > > 10. Sounds good.
> > > > >
> > > > > 13. Our current convention is to bump up the version of ZK value if 
> > > > > there is any format change. For example, we have bumped up the 
> > > > > version of the value in /brokers/ids/nnn multiple times and all of 
> > > > > those changes are compatible (just adding new fields). This has the 
> > > > > slight benefit that it makes it clear there is a format change. 
> > > > > Rolling upgrades and downgrades can still be supported with the 
> > > > > version bump. For example, if you
> > > > downgrade
> > > > > from a compatible change, you can leave the new format in ZK and the 
> > > > > old code will only pick up fields relevant to the old version. 
> > > > > Upgrade will
> > > > be
> > > > > controlled by inter broker protocol.
> > > >
> > > > Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> > > > protocol version.  We also need to return UnsupportedVersionException 
> > > > from the alterPartitionReassignments and listPartitionReassignments 
> > > > APIs when the IBP is too low.  This sounds doable, although we might 
> > > > need a release note that upgrading the IBP is necessary to allow 
> > > > reassignment operations after an upgrade.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Thanks for taking another look at this.
> > > > > >
> > > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > > Hi, Stan,
> > > > > > >
> > > > > > > Thanks for the explanation.
> > > > > > >
> > > > > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > > > > future
> > > > work,
> > > > > > > perhaps they should be added when we do the future work instead 
> > > > > > > of
> > > > now?
> > > > > >
> > > > > > I think this ties in with one of the big goals of this KIP, making 
> > > > > > it possible to distinguish reassigning replicas from normal replicas.
> > > > This is
> > > > > > the key to follow-on work like being able to ensure that 
> > > > > > partitions
> > > > with a
> > > > > > reassignment don't get falsely flagged as under-replicated in the
> > > > metrics,
> > > > > > or implementing reassignment quotas that don't accidentally affect
> > > > normal
> > > > > > replication traffic when a replica falls out of the ISR.
> > > > > >
> > > > > > For these follow-on improvements, we need to have that information 
> > > > > > in LeaderAndIsrRequest.  We could add the information in a 
> > > > > > follow-on KIP,
> > > > of
> > > > > > course, but then all the improvements are blocked on that 
> > > > > > follow-on
> > > > KIP.
> > > > > > That would slow things down for all of the downstream KIPs that 
> > > > > > are
> > > > blocked
> > > > > > on this.
> > > > > >
> > > > > > Also, to keep things consistent, I think it would be best if the
> > > > format of
> > > > > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > > > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > > > > I think
> > > > it
> > > > > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > > > > >
> > > > > > > > > Should we include those two fields in UpdateMetadata and
> > > > potentially
> > > > > > > > > Metadata requests too?
> > > > > >
> > > > > > We had some discussion earlier about how metadata responses to 
> > > > > > clients
> > > > are
> > > > > > getting too large, in part because they include a lot of 
> > > > > > information
> > > > that
> > > > > > most clients don't need (such as the ISR).  I think reassignment 
> > > > > > information definitely falls in the category of something a client
> > > > doesn't
> > > > > > need to know, so we shouldn't include it.
> > > > > >
> > > > > > A program like CruiseControl, or the command-line reassignment 
> > > > > > program, just wants to get the most up-to-date information about 
> > > > > > the state of reassigning partitions.  The MetadataRequest API 
> > > > > > wouldn't deliver that, because there are inherently delays in how 
> > > > > > we propagate metadata to brokers.  That's why the 
> > > > > > ListPartitionReassignments API is a better
> > > > choice
> > > > > > for those programs.  So I think if we added this information to 
> > > > > > the MetadataResponse, nobody would actually use it, and it would 
> > > > > > just use
> > > > up
> > > > > > more bandwidth.
> > > > > >
> > > > > > Of course, we can always revisit this later if we find a scenario
> > > > where a
> > > > > > producer or consumer would actually care about this.  But I think 
> > > > > > we
> > > > should
> > > > > > default to not adding stuff to the metadata response if we don't 
> > > > > > have a good use case in mind.
> > > > > >
> > > > > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > > > > we
> > > > > > cancel the
> > > > > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > > > > them
> > > > from
> > > > > > (the
> > > > > > > > > updated from the last-reassignment) R and TR, and starting
> > > > anew." In
> > > > > > this
> > > > > > > > > case, it seems that the controller needs to issue a 
> > > > > > > > > StopReplica
> > > > > > request to
> > > > > > > > > remove those unneeded replicas.
> > > > > >
> > > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > > >
> > > > > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > > > > AR
> > > > from
> > > > > > R,
> > > > > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> > > > cancel
> > > > > > the
> > > > > > > > > replica movements that have not yet completed." Similar to 
> > > > > > > > > the
> > > > > > above, it
> > > > > > > > > seems the controller needs to issue a StopReplica request to
> > > > remove
> > > > > > those
> > > > > > > > > unneeded replicas.
> > > > > >
> > > > > > Right.  Let's add this.
> > > > > >
> > > > > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> > > > should
> > > > > > we bump
> > > > > > > > > up the version number in the json value?
> > > > > >
> > > > > > The change to the zNode is backwards compatible, though.  Older 
> > > > > > brokers will continue to work, but just ignore the new fields.  If 
> > > > > > we bump that version number, then downgrades will require hand-editing zookeeper.
> > > > (Of
> > > > > > course downgrade isn't officially supported, but it would be nice 
> > > > > > not
> > > > to
> > > > > > break it if we don't need to...)  Changing the version number 
> > > > > > would
> > > > also
> > > > > > create problems during a rolling upgrade.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > > > > <cmccabe@apache.org
> > > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> > > > Robert
> > > > > > > > Barrett,
> > > > > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > > > > Jason
> > > > > > > > Gustafson,
> > > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > > sql_consulting@yahoo.com
> > > > > > > > > > .invalid>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > > George
> > > > > > > > > > > >
> > > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> > > > Shapira <
> > > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> > > > through
> > > > > > all
> > > > > > > > the
> > > > > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > > > > ended
> > > > up
> > > > > > with
> > > > > > > > a
> > > > > > > > > > > > great protocol.
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > > > > jason@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > > > > >
> > > > > > > > > > > > > -Jason
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > > > > cmccabe@apache.org
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > > > > reflect
> > > > the
> > > > > > fact
> > > > > > > > > that
> > > > > > > > > > > > we've
> > > > > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > > > > for 3
> > > > days
> > > > > > as
> > > > > > > > > > usual.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > > > > Kozlovski
> > > > wrote:
> > > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We have further iterated on the KIP in the
> > > > accompanying
> > > > > > > > > > discussion
> > > > > > > > > > > > thread
> > > > > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > > - we will internally use two collections to
> > > > represent a
> > > > > > > > > > reassignment
> > > > > > > > > > > > -
> > > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > > LeaderAndIsr has
> > > > > > > > been
> > > > > > > > > > > > updated
> > > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > > > > collection,
> > > > > > > > > > but
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > > > > collections
> > > > - the
> > > > > > > > full
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > set,
> > > > > > > > > > > > > > > the replicas we are adding as part of this
> > > > reassignment
> > > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > > > > proper
> > > > > > rollback
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > > > > API
> > > > call
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > https://cwik
> > > > i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > > C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> > > > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > > > > cmccabe@apache.org
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > > > > getting
> > > > a PR
> > > > > > > > done
> > > > > > > > > > as a
> > > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > > > > we'll
> > > > > > finish up
> > > > > > > > > the
> > > > > > > > > > > > vote.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > What is the estimated timeline to have this
> > > > > > implemented?
> > > > > > > > > If
> > > > > > > > > > any
> > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > > > > cancelling
> > > > > > > > > > reassignments,  I
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > > > > PDT,
> > > > Colin
> > > > > > > > McCabe
> > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> > > > individual
> > > > > > > > > > partitions to
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > > > > these
> > > > > > > > > operations
> > > > > > > > > > to
> > > > > > > > > > > > be at
> > > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > > > > good
> > > > > > > > > improvements
> > > > > > > > > > of
> > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > Cancellation
> > > > > > > > > > of
> > > > > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > > > > when
> > > > the
> > > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > > > > empty
> > > > > > replicas
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > > > > implementing
> > > > the
> > > > > > the
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> > > > it
> > > > > > will
> > > > > > > > > make
> > > > > > > > > > > > KIP-236
> > > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > > > > all
> > > > > > pending
> > > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > > one just needs to do a
> > > > > > ListPartitionRessignmentRequest,
> > > > > > > > > > then
> > > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > > > > topic/partitions
> > > > in
> > > > > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > > > > PDT,
> > > > Colin
> > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > > > > McCabe
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> > > > Gustafson
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > > > > the
> > > > > > partition
> > > > > > > > > > level
> > > > > > > > > > > > is
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > > > > understand
> > > > > > > > that
> > > > > > > > > > > > there is
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > reassignment in progress. This makes
> > > > KIP-352 a
> > > > > > > > > trivial
> > > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > > > > whether
> > > > the
> > > > > > > > leader
> > > > > > > > > > and
> > > > > > > > > > > > isr
> > > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > right place to store the target
> > > > reassignment.
> > > > > > It
> > > > > > > > is a
> > > > > > > > > > bit
> > > > > > > > > > > > odd
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > > > > from
> > > > the
> > > > > > > > > current
> > > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > > assume the thinking is probably that
> > > > although
> > > > > > the
> > > > > > > > > > current
> > > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > > > > znode as
> > > > > > well, it
> > > > > > > > > is
> > > > > > > > > > > > hard to
> > > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> > > > But
> > > > > > if we
> > > > > > > > > > have no
> > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > > the assignment znode, do you see a
> > > > downside to
> > > > > > > > > storing
> > > > > > > > > > the
> > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > > > > better
> > > > to
> > > > > > keep
> > > > > > > > the
> > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > > > > current
> > > > > > > > assignment,
> > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > > > > we
> > > > should
> > > > > > > > stick
> > > > > > > > > > with
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > > > > reassignment
> > > > state
> > > > > > into
> > > > > > > > > > > > > > > > > >
> > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > > The
> > > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > > > > epoch
> > > > for the
> > > > > > > > > > partition
> > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > > > > leader
> > > > > > epoch
> > > > > > > > > > resides
> > > > > > > > > > > > in
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > znode anyway.  I agree there is some
> > > > inconsistency
> > > > > > > > here,
> > > > > > > > > > but
> > > > > > > > > > > > so be
> > > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > > > > data
> > > > > > > > structures,
> > > > > > > > > > we
> > > > > > > > > > > > might
> > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > > > > will
> > > > work
> > > > > > fine
> > > > > > > > > and
> > > > > > > > > > be
> > > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > > > > altering,
> > > > > > right?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > > > > reads
> > > > a
> > > > > > little
> > > > > > > > > > > > better, and
> > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > > > > +the
> > > > wiki.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > > > > Metadata
> > > > API? In
> > > > > > > > other
> > > > > > > > > > > > words,
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> > > > probably
> > > > > > > > need
> > > > > > > > > a
> > > > > > > > > > > > change
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> > > > alternative I
> > > > > > can
> > > > > > > > > > think of
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > request
> > > > > > > > as
> > > > > > > > > > the
> > > > > > > > > > > > union
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > > > > can't
> > > > think
> > > > > > of
> > > > > > > > any
> > > > > > > > > > > > benefit
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > > > > this, we
> > > > > > > > probably
> > > > > > > > > > > > wouldn't
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > > > > think on
> > > > > > balance,
> > > > > > > > > > you're
> > > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > > > > with
> > > > the
> > > > > > > > replica
> > > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > > > > information is
> > > > > > > > > > > > available in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > > > > this,
> > > > we'll
> > > > > > need
> > > > > > > > a
> > > > > > > > > > flag
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > > > > show me
> > > > > > > > > reassigning
> > > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > > > > stick
> > > > with
> > > > > > the
> > > > > > > > > > original
> > > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > > ListPartitionReassignments
> > > > > > > > > > API.
> > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > > > > or
> > > > > > Consumer
> > > > > > > > > needs
> > > > > > > > > > to
> > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > about a reassignment.  They should just be
> > > > notified
> > > > > > > > when
> > > > > > > > > > the
> > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > partitions changes, which doesn't require
> > > > changes
> > > > > > to
> > > > > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > > > > client
> > > > only
> > > > > > cares
> > > > > > > > if
> > > > > > > > > > > > someone
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > managing the reassignment.  So adding this
> > > > state
> > > > > > to the
> > > > > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > > > > benefit.  In
> > > > > > > > > the
> > > > > > > > > > > > common
> > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> > > > would
> > > > > > be 4
> > > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > In general, I think we have a problem of
> > > > > > oversharing in
> > > > > > > > > the
> > > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > > > > 100x
> > > > the
> > > > > > number
> > > > > > > > > of
> > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > > > > about
> > > > giving
> > > > > > > > > clients
> > > > > > > > > > > > only
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > information they actually need, about the
> > > > > > partitions
> > > > > > > > they
> > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > > > > in the
> > > > > > > > category
> > > > > > > > > of
> > > > > > > > > > > > state
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > isn't needed by clients (except very
> > > > specialized
> > > > > > > > > > rebalancing
> > > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > > > > that
> > > > > > someone
> > > > > > > > > > managing
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > > information,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > > > > adding
> > > > this
> > > > > > > > state
> > > > > > > > > to
> > > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > > or describeTopics, which could contact any
> > > > node in
> > > > > > the
> > > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > > > > describeTopics
> > > > > > > > > feels
> > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > > > > interacts
> > > > with
> > > > > > other
> > > > > > > > > > extra
> > > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > > > > I
> > > > think
> > > > > > a
> > > > > > > > > > separate
> > > > > > > > > > > > API
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > > > > will
> > > > join
> > > > > > the
> > > > > > > > > ISR.
> > > > > > > > > > > > Will we
> > > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > > target replicas joining the ISR before
> > > > taking
> > > > > > the
> > > > > > > > > > replica
> > > > > > > > > > > > out
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > > > > target
> > > > > > replicas
> > > > > > > > can
> > > > > > > > > > > > still be
> > > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We'll take a replica out of the target
> > > > replicas
> > > > > > set
> > > > > > > > as
> > > > > > > > > > soon
> > > > > > > > > > > > as
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > > > > this
> > > > in
> > > > > > the
> > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > > > > permissions
> > > > for
> > > > > > the
> > > > > > > > new
> > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Good point.  I think
> > > > alterPartitionAssignments
> > > > > > should
> > > > > > > > > > require
> > > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > > > > will be
> > > > > > > > > unchanged.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> > > > Shapira <
> > > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > > > > have
> > > > > > this new
> > > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > > > > Colin
> > > > McCabe
> > > > > > <
> > > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > > > > Create
> > > > > > > > an
> > > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > > > > this
> > > > KIP is
> > > > > > > > > > important
> > > > > > > > > > > > since
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > > reassignment
> > > > > > > > > > (see
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > > > > discussions
> > > > > > > > > we've
> > > > > > > > > > had
> > > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > important
> > > > > > > > > > KIP-4
> > > > > > > > > > > > goal
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > > > > discussion in
> > > > > > > > > the
> > > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > explicitly
> > > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > different
> > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > > > > simplified
> > > > the
> > > > > > API
> > > > > > > > a
> > > > > > > > > > bit by
> > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > > > > is
> > > > used
> > > > > > by
> > > > > > > > both
> > > > > > > > > > the
> > > > > > > > > > > > alter
> > > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > deprecate
> > > > > > > > > > the
> > > > > > > > > > > > old
> > > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > > rather than removing it completely.
> > > > That
> > > > > > > > should
> > > > > > > > > > give
> > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > > > > the
> > > > new
> > > > > > API.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > > added a
> > > > > > > > > note
> > > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > > > > now
> > > > use a
> > > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > > https:
> > > > > > > > > %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > > served=0>
> > > > > > > > > > |
> > > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnam06.safelinks.protection&amp;data=02%7C01%7Ckchitta%40microsoft.com%7C3e662dd1d60949f2ff6808d71b95e68f%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637008202957720437&amp;sdata=vdDj2GPhjVP5TQj1B4%2BoCjGU6cgDqGVYOEnqaMurz68%3D&amp;reserved=0.
> > > > > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > > 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > --
> > > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > 
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Aug 7, 2019, at 15:41, George Li wrote:
> This email seemed to get lost in the dev email server.  Resending. 
> 
> 
> On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li 
> <sq...@yahoo.com> wrote:
> 
> 
> The pending reassignments partitions would be reported as URP (Under 
> Replicated Partitions).  or maybe reported as a separate metrics of 
> RURP (Reassignment URP) since now we can derived from the new 
> AddingReplicas. An alert could be triggered based on this. 
> 

Hi George,

I agree that this would be a great idea for follow up work.  Check out KIP-352, which discusses creating a such a metric. :)

> 
> 
> It would be nice if ListPartitionReassignmentResult could return the 
> "elapsed time/duration" of the current pending reassignments, the 
> calling client can flag those current long running reassignments and 
> alert.  However, what I would be interested is probably the total # of 
> pending reassignments because I will submit reassignments in batches, 
> e.g. 50 reassignments per batch.  If the pending reassignments # is 
> below that per batch #, submit more new reassignments = (per_batch_# - 
> pending_#).
> 

It is definitely useful to know what reassignments exist.  If you call ListPartitionReassignments, you can count how many results you get, in order to implement a policy like that.

I'm not sure if knowing how long reassignments have been in progress will be important or not.  I think we should give people some time to try out the new APIs and see what could be improved based on their experience.

> 
> 
> It seems currently, the ReplicaFetcher threads could quite easily crash 
> because of some exceptions. e.g. Java Out Of Memory, and would just 
> remain dead (jstack to dump threads to check the # of running 
> ReplicaFetcher threads) without getting restarted automatically, so 
> needs to bounce the broker.  It would be nice to make the 
> ReplicaFetcher more robust/resilient of catching more exceptions, and 
> if crashed, get restarted after some time. 
> 

This has definitely been an issue in the past, I agree.  Thankfully, we recently did improve the robustness of the ReplicaFetcher.  Check out "KIP-461: Improve Replica Fetcher behavior at handling partition failure."

cheers,
Colin

> 
> 
> Thanks,
> 
> George
> 
> 
> 
> On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote: 
> > Hi Koushik,
> > 
> > Thanks for the idea.  This KIP is already pretty big, so I think we'll have to consider ideas like this in follow-on KIPs.
> > 
> > In general, figuring out what's wrong with replication is a pretty tough problem.  If we had an API for this, we'd probably want it to be unified, and not specific to reassigning partitions.
> > 
> > regards,
> > Colin
> > 
> > 
> > On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > > Hey Colin,
> > > 
> > > Can the ListPartitionReassignmentsResult include the status of the 
> > > current reassignment progress of each partition? A reassignment can be 
> > > in progress for different reasons and the status can give the option to 
> > > alter the current reassignment.
> > > 
> > > Example -  A leaderISRRequest of a new assigned replicas can be 
> > > ignored/errored because of a storage exception.  And reassignment batch 
> > > will be waiting indefinitely for the new assigned replicas to be in 
> > > sync with the leader of the partition.  
> > > 	      Showing the status will give an option to alter the affected 
> > > partitions and allow the batch to complete reassignment.
> > > 
> > > OAR = {1, 2, 3} and RAR = {4,5,6}
> > > 
> > >      AR                 leader/isr
> > >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest 
> > > was lost/skipped for 5 and the reassignment operation will be waiting 
> > > indefinitely for the 5 to be insync.
> > > 
> > > 
> > > 
> > > Thanks,
> > > Koushik
> > > 
> > > -----Original Message-----
> > > From: Jun Rao <ju...@confluent.io> 
> > > Sent: Friday, August 2, 2019 10:04 AM
> > > To: dev <de...@kafka.apache.org>
> > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > > Reassignment
> > > 
> > > Hi, Colin,
> > > 
> > > First, since we are changing the format of LeaderAndIsrRequest, which 
> > > is an inter broker request, it seems that we will need IBP during 
> > > rolling upgrade. Could we add that to the compatibility section?
> > > 
> > > Regarding UnsupportedVersionException, even without ZK node version 
> > > bump, we probably want to only use the new ZK value fields after all 
> > > brokers have been upgraded to the new binary. Otherwise, the 
> > > reassignment task may not be completed if the controller changes to a 
> > > broker still on the old binary.
> > > IBP is one way to achieve that. The main thing is that we need some way 
> > > for the controller to deal with the new ZK fields. Dealing with the 
> > > additional ZK node version bump seems a small thing on top of that?
> > > 
> > > Thanks,
> > > 
> > > Jun
> > > 
> > > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:
> > > 
> > > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > > Hi, Colin,
> > > > >
> > > > > 10. Sounds good.
> > > > >
> > > > > 13. Our current convention is to bump up the version of ZK value if 
> > > > > there is any format change. For example, we have bumped up the 
> > > > > version of the value in /brokers/ids/nnn multiple times and all of 
> > > > > those changes are compatible (just adding new fields). This has the 
> > > > > slight benefit that it makes it clear there is a format change. 
> > > > > Rolling upgrades and downgrades can still be supported with the 
> > > > > version bump. For example, if you
> > > > downgrade
> > > > > from a compatible change, you can leave the new format in ZK and the 
> > > > > old code will only pick up fields relevant to the old version. 
> > > > > Upgrade will
> > > > be
> > > > > controlled by inter broker protocol.
> > > >
> > > > Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> > > > protocol version.  We also need to return UnsupportedVersionException 
> > > > from the alterPartitionReassignments and listPartitionReassignments 
> > > > APIs when the IBP is too low.  This sounds doable, although we might 
> > > > need a release note that upgrading the IBP is necessary to allow 
> > > > reassignment operations after an upgrade.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Thanks for taking another look at this.
> > > > > >
> > > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > > Hi, Stan,
> > > > > > >
> > > > > > > Thanks for the explanation.
> > > > > > >
> > > > > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > > > > future
> > > > work,
> > > > > > > perhaps they should be added when we do the future work instead 
> > > > > > > of
> > > > now?
> > > > > >
> > > > > > I think this ties in with one of the big goals of this KIP, making 
> > > > > > it possible to distinguish reassigning replicas from normal replicas.
> > > > This is
> > > > > > the key to follow-on work like being able to ensure that 
> > > > > > partitions
> > > > with a
> > > > > > reassignment don't get falsely flagged as under-replicated in the
> > > > metrics,
> > > > > > or implementing reassignment quotas that don't accidentally affect
> > > > normal
> > > > > > replication traffic when a replica falls out of the ISR.
> > > > > >
> > > > > > For these follow-on improvements, we need to have that information 
> > > > > > in LeaderAndIsrRequest.  We could add the information in a 
> > > > > > follow-on KIP,
> > > > of
> > > > > > course, but then all the improvements are blocked on that 
> > > > > > follow-on
> > > > KIP.
> > > > > > That would slow things down for all of the downstream KIPs that 
> > > > > > are
> > > > blocked
> > > > > > on this.
> > > > > >
> > > > > > Also, to keep things consistent, I think it would be best if the
> > > > format of
> > > > > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > > > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > > > > I think
> > > > it
> > > > > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > > > > >
> > > > > > > > > Should we include those two fields in UpdateMetadata and
> > > > potentially
> > > > > > > > > Metadata requests too?
> > > > > >
> > > > > > We had some discussion earlier about how metadata responses to 
> > > > > > clients
> > > > are
> > > > > > getting too large, in part because they include a lot of 
> > > > > > information
> > > > that
> > > > > > most clients don't need (such as the ISR).  I think reassignment 
> > > > > > information definitely falls in the category of something a client
> > > > doesn't
> > > > > > need to know, so we shouldn't include it.
> > > > > >
> > > > > > A program like CruiseControl, or the command-line reassignment 
> > > > > > program, just wants to get the most up-to-date information about 
> > > > > > the state of reassigning partitions.  The MetadataRequest API 
> > > > > > wouldn't deliver that, because there are inherently delays in how 
> > > > > > we propagate metadata to brokers.  That's why the 
> > > > > > ListPartitionReassignments API is a better
> > > > choice
> > > > > > for those programs.  So I think if we added this information to 
> > > > > > the MetadataResponse, nobody would actually use it, and it would 
> > > > > > just use
> > > > up
> > > > > > more bandwidth.
> > > > > >
> > > > > > Of course, we can always revisit this later if we find a scenario
> > > > where a
> > > > > > producer or consumer would actually care about this.  But I think 
> > > > > > we
> > > > should
> > > > > > default to not adding stuff to the metadata response if we don't 
> > > > > > have a good use case in mind.
> > > > > >
> > > > > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > > > > we
> > > > > > cancel the
> > > > > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > > > > them
> > > > from
> > > > > > (the
> > > > > > > > > updated from the last-reassignment) R and TR, and starting
> > > > anew." In
> > > > > > this
> > > > > > > > > case, it seems that the controller needs to issue a 
> > > > > > > > > StopReplica
> > > > > > request to
> > > > > > > > > remove those unneeded replicas.
> > > > > >
> > > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > > >
> > > > > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > > > > AR
> > > > from
> > > > > > R,
> > > > > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> > > > cancel
> > > > > > the
> > > > > > > > > replica movements that have not yet completed." Similar to 
> > > > > > > > > the
> > > > > > above, it
> > > > > > > > > seems the controller needs to issue a StopReplica request to
> > > > remove
> > > > > > those
> > > > > > > > > unneeded replicas.
> > > > > >
> > > > > > Right.  Let's add this.
> > > > > >
> > > > > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> > > > should
> > > > > > we bump
> > > > > > > > > up the version number in the json value?
> > > > > >
> > > > > > The change to the zNode is backwards compatible, though.  Older 
> > > > > > brokers will continue to work, but just ignore the new fields.  If 
> > > > > > we bump that version number, then downgrades will require hand-editing zookeeper.
> > > > (Of
> > > > > > course downgrade isn't officially supported, but it would be nice 
> > > > > > not
> > > > to
> > > > > > break it if we don't need to...)  Changing the version number 
> > > > > > would
> > > > also
> > > > > > create problems during a rolling upgrade.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > > > > <cmccabe@apache.org
> > > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> > > > Robert
> > > > > > > > Barrett,
> > > > > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > > > > Jason
> > > > > > > > Gustafson,
> > > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > > sql_consulting@yahoo.com
> > > > > > > > > > .invalid>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > > George
> > > > > > > > > > > >
> > > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> > > > Shapira <
> > > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> > > > through
> > > > > > all
> > > > > > > > the
> > > > > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > > > > ended
> > > > up
> > > > > > with
> > > > > > > > a
> > > > > > > > > > > > great protocol.
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > > > > jason@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > > > > >
> > > > > > > > > > > > > -Jason
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > > > > cmccabe@apache.org
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > > > > reflect
> > > > the
> > > > > > fact
> > > > > > > > > that
> > > > > > > > > > > > we've
> > > > > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > > > > for 3
> > > > days
> > > > > > as
> > > > > > > > > > usual.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > > > > Kozlovski
> > > > wrote:
> > > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We have further iterated on the KIP in the
> > > > accompanying
> > > > > > > > > > discussion
> > > > > > > > > > > > thread
> > > > > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > > - we will internally use two collections to
> > > > represent a
> > > > > > > > > > reassignment
> > > > > > > > > > > > -
> > > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > > LeaderAndIsr has
> > > > > > > > been
> > > > > > > > > > > > updated
> > > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > > > > collection,
> > > > > > > > > > but
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > > > > collections
> > > > - the
> > > > > > > > full
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > set,
> > > > > > > > > > > > > > > the replicas we are adding as part of this
> > > > reassignment
> > > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > > > > proper
> > > > > > rollback
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > > > > API
> > > > call
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwik
> > > > i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > > C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> > > > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > > > > cmccabe@apache.org
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > > > > getting
> > > > a PR
> > > > > > > > done
> > > > > > > > > > as a
> > > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > > > > we'll
> > > > > > finish up
> > > > > > > > > the
> > > > > > > > > > > > vote.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > What is the estimated timeline to have this
> > > > > > implemented?
> > > > > > > > > If
> > > > > > > > > > any
> > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > > > > cancelling
> > > > > > > > > > reassignments,  I
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > > > > PDT,
> > > > Colin
> > > > > > > > McCabe
> > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> > > > individual
> > > > > > > > > > partitions to
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > > > > these
> > > > > > > > > operations
> > > > > > > > > > to
> > > > > > > > > > > > be at
> > > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > > > > good
> > > > > > > > > improvements
> > > > > > > > > > of
> > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > Cancellation
> > > > > > > > > > of
> > > > > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > > > > when
> > > > the
> > > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > > > > empty
> > > > > > replicas
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > > > > implementing
> > > > the
> > > > > > the
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> > > > it
> > > > > > will
> > > > > > > > > make
> > > > > > > > > > > > KIP-236
> > > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > > > > all
> > > > > > pending
> > > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > > one just needs to do a
> > > > > > ListPartitionRessignmentRequest,
> > > > > > > > > > then
> > > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > > > > topic/partitions
> > > > in
> > > > > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > > > > PDT,
> > > > Colin
> > > > > > > > McCabe
> > > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > > > > McCabe
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> > > > Gustafson
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > > > > the
> > > > > > partition
> > > > > > > > > > level
> > > > > > > > > > > > is
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > > > > understand
> > > > > > > > that
> > > > > > > > > > > > there is
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > reassignment in progress. This makes
> > > > KIP-352 a
> > > > > > > > > trivial
> > > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > > > > whether
> > > > the
> > > > > > > > leader
> > > > > > > > > > and
> > > > > > > > > > > > isr
> > > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > right place to store the target
> > > > reassignment.
> > > > > > It
> > > > > > > > is a
> > > > > > > > > > bit
> > > > > > > > > > > > odd
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > > > > from
> > > > the
> > > > > > > > > current
> > > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > > assume the thinking is probably that
> > > > although
> > > > > > the
> > > > > > > > > > current
> > > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > > > > znode as
> > > > > > well, it
> > > > > > > > > is
> > > > > > > > > > > > hard to
> > > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> > > > But
> > > > > > if we
> > > > > > > > > > have no
> > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > > the assignment znode, do you see a
> > > > downside to
> > > > > > > > > storing
> > > > > > > > > > the
> > > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > > > > better
> > > > to
> > > > > > keep
> > > > > > > > the
> > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > > > > current
> > > > > > > > assignment,
> > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > > > > we
> > > > should
> > > > > > > > stick
> > > > > > > > > > with
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > > > > reassignment
> > > > state
> > > > > > into
> > > > > > > > > > > > > > > > > >
> > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > > The
> > > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > > > > epoch
> > > > for the
> > > > > > > > > > partition
> > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > > > > leader
> > > > > > epoch
> > > > > > > > > > resides
> > > > > > > > > > > > in
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > znode anyway.  I agree there is some
> > > > inconsistency
> > > > > > > > here,
> > > > > > > > > > but
> > > > > > > > > > > > so be
> > > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > > > > data
> > > > > > > > structures,
> > > > > > > > > > we
> > > > > > > > > > > > might
> > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > > > > will
> > > > work
> > > > > > fine
> > > > > > > > > and
> > > > > > > > > > be
> > > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > > > > altering,
> > > > > > right?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > > > > reads
> > > > a
> > > > > > little
> > > > > > > > > > > > better, and
> > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > > > > +the
> > > > wiki.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > > > > Metadata
> > > > API? In
> > > > > > > > other
> > > > > > > > > > > > words,
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> > > > probably
> > > > > > > > need
> > > > > > > > > a
> > > > > > > > > > > > change
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> > > > alternative I
> > > > > > can
> > > > > > > > > > think of
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > > > > Metadata
> > > > > > request
> > > > > > > > as
> > > > > > > > > > the
> > > > > > > > > > > > union
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > > > > can't
> > > > think
> > > > > > of
> > > > > > > > any
> > > > > > > > > > > > benefit
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > > > > this, we
> > > > > > > > probably
> > > > > > > > > > > > wouldn't
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > > > > think on
> > > > > > balance,
> > > > > > > > > > you're
> > > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > > > > with
> > > > the
> > > > > > > > replica
> > > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > > > > information is
> > > > > > > > > > > > available in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > > > > this,
> > > > we'll
> > > > > > need
> > > > > > > > a
> > > > > > > > > > flag
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > > > > show me
> > > > > > > > > reassigning
> > > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > > > > stick
> > > > with
> > > > > > the
> > > > > > > > > > original
> > > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > > ListPartitionReassignments
> > > > > > > > > > API.
> > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > > > > or
> > > > > > Consumer
> > > > > > > > > needs
> > > > > > > > > > to
> > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > about a reassignment.  They should just be
> > > > notified
> > > > > > > > when
> > > > > > > > > > the
> > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > partitions changes, which doesn't require
> > > > changes
> > > > > > to
> > > > > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > > > > client
> > > > only
> > > > > > cares
> > > > > > > > if
> > > > > > > > > > > > someone
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > managing the reassignment.  So adding this
> > > > state
> > > > > > to the
> > > > > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > > > > benefit.  In
> > > > > > > > > the
> > > > > > > > > > > > common
> > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> > > > would
> > > > > > be 4
> > > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > In general, I think we have a problem of
> > > > > > oversharing in
> > > > > > > > > the
> > > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > > > > 100x
> > > > the
> > > > > > number
> > > > > > > > > of
> > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > > > > about
> > > > giving
> > > > > > > > > clients
> > > > > > > > > > > > only
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > information they actually need, about the
> > > > > > partitions
> > > > > > > > they
> > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > > > > in the
> > > > > > > > category
> > > > > > > > > of
> > > > > > > > > > > > state
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > isn't needed by clients (except very
> > > > specialized
> > > > > > > > > > rebalancing
> > > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > > > > that
> > > > > > someone
> > > > > > > > > > managing
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > > information,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > > > > adding
> > > > this
> > > > > > > > state
> > > > > > > > > to
> > > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > > or describeTopics, which could contact any
> > > > node in
> > > > > > the
> > > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > > > > describeTopics
> > > > > > > > > feels
> > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > > > > interacts
> > > > with
> > > > > > other
> > > > > > > > > > extra
> > > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > > > > I
> > > > think
> > > > > > a
> > > > > > > > > > separate
> > > > > > > > > > > > API
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > > > > will
> > > > join
> > > > > > the
> > > > > > > > > ISR.
> > > > > > > > > > > > Will we
> > > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > > target replicas joining the ISR before
> > > > taking
> > > > > > the
> > > > > > > > > > replica
> > > > > > > > > > > > out
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > > > > target
> > > > > > replicas
> > > > > > > > can
> > > > > > > > > > > > still be
> > > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We'll take a replica out of the target
> > > > replicas
> > > > > > set
> > > > > > > > as
> > > > > > > > > > soon
> > > > > > > > > > > > as
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > > > > this
> > > > in
> > > > > > the
> > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > > > > permissions
> > > > for
> > > > > > the
> > > > > > > > new
> > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Good point.  I think
> > > > alterPartitionAssignments
> > > > > > should
> > > > > > > > > > require
> > > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > > > > will be
> > > > > > > > > unchanged.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> > > > Shapira <
> > > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > > > > have
> > > > > > this new
> > > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > > > > Colin
> > > > McCabe
> > > > > > <
> > > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > > > > Create
> > > > > > > > an
> > > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > > > > this
> > > > KIP is
> > > > > > > > > > important
> > > > > > > > > > > > since
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > > reassignment
> > > > > > > > > > (see
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > > > > discussions
> > > > > > > > > we've
> > > > > > > > > > had
> > > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > important
> > > > > > > > > > KIP-4
> > > > > > > > > > > > goal
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > > > > discussion in
> > > > > > > > > the
> > > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > explicitly
> > > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > different
> > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > > > > simplified
> > > > the
> > > > > > API
> > > > > > > > a
> > > > > > > > > > bit by
> > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > > > > is
> > > > used
> > > > > > by
> > > > > > > > both
> > > > > > > > > > the
> > > > > > > > > > > > alter
> > > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > > > > now
> > > > > > > > deprecate
> > > > > > > > > > the
> > > > > > > > > > > > old
> > > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > > rather than removing it completely.
> > > > That
> > > > > > > > should
> > > > > > > > > > give
> > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > > > > the
> > > > new
> > > > > > API.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > > added a
> > > > > > > > > note
> > > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > > > > now
> > > > use a
> > > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > > https://nam06.safelinks.protection.outlook.com/?url=https%3A
> > > > > > > > > %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > > served=0>
> > > > > > > > > > |
> > > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.
> > > > > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > > 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > --
> > > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > 
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by George Li <sq...@gmail.com>.
This email seemed to get lost in the dev email server.  Resending. 


On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li <sq...@yahoo.com> wrote:


The pending reassignments partitions would be reported as URP (Under Replicated Partitions).  or maybe reported as a separate metrics of RURP (Reassignment URP) since now we can derived from the new AddingReplicas. An alert could be triggered based on this. 



It would be nice if ListPartitionReassignmentResult could return the "elapsed time/duration" of the current pending reassignments, the calling client can flag those current long running reassignments and alert.  However, what I would be interested is probably the total # of pending reassignments because I will submit reassignments in batches, e.g. 50 reassignments per batch.  If the pending reassignments # is below that per batch #, submit more new reassignments = (per_batch_# - pending_#).



It seems currently, the ReplicaFetcher threads could quite easily crash because of some exceptions. e.g. Java Out Of Memory, and would just remain dead (jstack to dump threads to check the # of running ReplicaFetcher threads) without getting restarted automatically, so needs to bounce the broker.  It would be nice to make the ReplicaFetcher more robust/resilient of catching more exceptions, and if crashed, get restarted after some time. 



Thanks,

George



On 2019/08/06 23:07:19, "Colin McCabe" <cm...@apache.org> wrote: 
> Hi Koushik,
> 
> Thanks for the idea.  This KIP is already pretty big, so I think we'll have to consider ideas like this in follow-on KIPs.
> 
> In general, figuring out what's wrong with replication is a pretty tough problem.  If we had an API for this, we'd probably want it to be unified, and not specific to reassigning partitions.
> 
> regards,
> Colin
> 
> 
> On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> > Hey Colin,
> > 
> > Can the ListPartitionReassignmentsResult include the status of the 
> > current reassignment progress of each partition? A reassignment can be 
> > in progress for different reasons and the status can give the option to 
> > alter the current reassignment.
> > 
> > Example -  A leaderISRRequest of a new assigned replicas can be 
> > ignored/errored because of a storage exception.  And reassignment batch 
> > will be waiting indefinitely for the new assigned replicas to be in 
> > sync with the leader of the partition.  
> > 	      Showing the status will give an option to alter the affected 
> > partitions and allow the batch to complete reassignment.
> > 
> > OAR = {1, 2, 3} and RAR = {4,5,6}
> > 
> >      AR                 leader/isr
> >     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest 
> > was lost/skipped for 5 and the reassignment operation will be waiting 
> > indefinitely for the 5 to be insync.
> > 
> > 
> > 
> > Thanks,
> > Koushik
> > 
> > -----Original Message-----
> > From: Jun Rao <ju...@confluent.io> 
> > Sent: Friday, August 2, 2019 10:04 AM
> > To: dev <de...@kafka.apache.org>
> > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > Reassignment
> > 
> > Hi, Colin,
> > 
> > First, since we are changing the format of LeaderAndIsrRequest, which 
> > is an inter broker request, it seems that we will need IBP during 
> > rolling upgrade. Could we add that to the compatibility section?
> > 
> > Regarding UnsupportedVersionException, even without ZK node version 
> > bump, we probably want to only use the new ZK value fields after all 
> > brokers have been upgraded to the new binary. Otherwise, the 
> > reassignment task may not be completed if the controller changes to a 
> > broker still on the old binary.
> > IBP is one way to achieve that. The main thing is that we need some way 
> > for the controller to deal with the new ZK fields. Dealing with the 
> > additional ZK node version bump seems a small thing on top of that?
> > 
> > Thanks,
> > 
> > Jun
> > 
> > On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:
> > 
> > > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > > Hi, Colin,
> > > >
> > > > 10. Sounds good.
> > > >
> > > > 13. Our current convention is to bump up the version of ZK value if 
> > > > there is any format change. For example, we have bumped up the 
> > > > version of the value in /brokers/ids/nnn multiple times and all of 
> > > > those changes are compatible (just adding new fields). This has the 
> > > > slight benefit that it makes it clear there is a format change. 
> > > > Rolling upgrades and downgrades can still be supported with the 
> > > > version bump. For example, if you
> > > downgrade
> > > > from a compatible change, you can leave the new format in ZK and the 
> > > > old code will only pick up fields relevant to the old version. 
> > > > Upgrade will
> > > be
> > > > controlled by inter broker protocol.
> > >
> > > Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> > > protocol version.  We also need to return UnsupportedVersionException 
> > > from the alterPartitionReassignments and listPartitionReassignments 
> > > APIs when the IBP is too low.  This sounds doable, although we might 
> > > need a release note that upgrading the IBP is necessary to allow 
> > > reassignment operations after an upgrade.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Thanks for taking another look at this.
> > > > >
> > > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > > Hi, Stan,
> > > > > >
> > > > > > Thanks for the explanation.
> > > > > >
> > > > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > > > future
> > > work,
> > > > > > perhaps they should be added when we do the future work instead 
> > > > > > of
> > > now?
> > > > >
> > > > > I think this ties in with one of the big goals of this KIP, making 
> > > > > it possible to distinguish reassigning replicas from normal replicas.
> > > This is
> > > > > the key to follow-on work like being able to ensure that 
> > > > > partitions
> > > with a
> > > > > reassignment don't get falsely flagged as under-replicated in the
> > > metrics,
> > > > > or implementing reassignment quotas that don't accidentally affect
> > > normal
> > > > > replication traffic when a replica falls out of the ISR.
> > > > >
> > > > > For these follow-on improvements, we need to have that information 
> > > > > in LeaderAndIsrRequest.  We could add the information in a 
> > > > > follow-on KIP,
> > > of
> > > > > course, but then all the improvements are blocked on that 
> > > > > follow-on
> > > KIP.
> > > > > That would slow things down for all of the downstream KIPs that 
> > > > > are
> > > blocked
> > > > > on this.
> > > > >
> > > > > Also, to keep things consistent, I think it would be best if the
> > > format of
> > > > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > > > I think
> > > it
> > > > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > > > >
> > > > > > > > Should we include those two fields in UpdateMetadata and
> > > potentially
> > > > > > > > Metadata requests too?
> > > > >
> > > > > We had some discussion earlier about how metadata responses to 
> > > > > clients
> > > are
> > > > > getting too large, in part because they include a lot of 
> > > > > information
> > > that
> > > > > most clients don't need (such as the ISR).  I think reassignment 
> > > > > information definitely falls in the category of something a client
> > > doesn't
> > > > > need to know, so we shouldn't include it.
> > > > >
> > > > > A program like CruiseControl, or the command-line reassignment 
> > > > > program, just wants to get the most up-to-date information about 
> > > > > the state of reassigning partitions.  The MetadataRequest API 
> > > > > wouldn't deliver that, because there are inherently delays in how 
> > > > > we propagate metadata to brokers.  That's why the 
> > > > > ListPartitionReassignments API is a better
> > > choice
> > > > > for those programs.  So I think if we added this information to 
> > > > > the MetadataResponse, nobody would actually use it, and it would 
> > > > > just use
> > > up
> > > > > more bandwidth.
> > > > >
> > > > > Of course, we can always revisit this later if we find a scenario
> > > where a
> > > > > producer or consumer would actually care about this.  But I think 
> > > > > we
> > > should
> > > > > default to not adding stuff to the metadata response if we don't 
> > > > > have a good use case in mind.
> > > > >
> > > > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > > > we
> > > > > cancel the
> > > > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > > > them
> > > from
> > > > > (the
> > > > > > > > updated from the last-reassignment) R and TR, and starting
> > > anew." In
> > > > > this
> > > > > > > > case, it seems that the controller needs to issue a 
> > > > > > > > StopReplica
> > > > > request to
> > > > > > > > remove those unneeded replicas.
> > > > >
> > > > > Good catch.  Yes, we should document this in the  KIP.
> > > > >
> > > > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > > > AR
> > > from
> > > > > R,
> > > > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> > > cancel
> > > > > the
> > > > > > > > replica movements that have not yet completed." Similar to 
> > > > > > > > the
> > > > > above, it
> > > > > > > > seems the controller needs to issue a StopReplica request to
> > > remove
> > > > > those
> > > > > > > > unneeded replicas.
> > > > >
> > > > > Right.  Let's add this.
> > > > >
> > > > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> > > should
> > > > > we bump
> > > > > > > > up the version number in the json value?
> > > > >
> > > > > The change to the zNode is backwards compatible, though.  Older 
> > > > > brokers will continue to work, but just ignore the new fields.  If 
> > > > > we bump that version number, then downgrades will require hand-editing zookeeper.
> > > (Of
> > > > > course downgrade isn't officially supported, but it would be nice 
> > > > > not
> > > to
> > > > > break it if we don't need to...)  Changing the version number 
> > > > > would
> > > also
> > > > > create problems during a rolling upgrade.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > > > <cmccabe@apache.org
> > > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> > > Robert
> > > > > > > Barrett,
> > > > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > > > Jason
> > > > > > > Gustafson,
> > > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > > >
> > > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > > sql_consulting@yahoo.com
> > > > > > > > > .invalid>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > > George
> > > > > > > > > > >
> > > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> > > Shapira <
> > > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > > >
> > > > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> > > through
> > > > > all
> > > > > > > the
> > > > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > > > ended
> > > up
> > > > > with
> > > > > > > a
> > > > > > > > > > > great protocol.
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > > > jason@confluent.io>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > > > >
> > > > > > > > > > > > -Jason
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > > > cmccabe@apache.org
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > > > reflect
> > > the
> > > > > fact
> > > > > > > > that
> > > > > > > > > > > we've
> > > > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > > > for 3
> > > days
> > > > > as
> > > > > > > > > usual.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > > > Kozlovski
> > > wrote:
> > > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We have further iterated on the KIP in the
> > > accompanying
> > > > > > > > > discussion
> > > > > > > > > > > thread
> > > > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > > - we will internally use two collections to
> > > represent a
> > > > > > > > > reassignment
> > > > > > > > > > > -
> > > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > > LeaderAndIsr has
> > > > > > > been
> > > > > > > > > > > updated
> > > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > > > collection,
> > > > > > > > > but
> > > > > > > > > > > the
> > > > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > > > collections
> > > - the
> > > > > > > full
> > > > > > > > > > > replica
> > > > > > > > > > > > > set,
> > > > > > > > > > > > > > the replicas we are adding as part of this
> > > reassignment
> > > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > > > proper
> > > > > rollback
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > > > API
> > > call
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwik
> > > i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&data=02%7
> > > C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> > > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&sdata=w
> > > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&reserved=0
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > > > cmccabe@apache.org
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > > > getting
> > > a PR
> > > > > > > done
> > > > > > > > > as a
> > > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > > > we'll
> > > > > finish up
> > > > > > > > the
> > > > > > > > > > > vote.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > > > (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > What is the estimated timeline to have this
> > > > > implemented?
> > > > > > > > If
> > > > > > > > > any
> > > > > > > > > > > help
> > > > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > > > cancelling
> > > > > > > > > reassignments,  I
> > > > > > > > > > > can
> > > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > > > PDT,
> > > Colin
> > > > > > > McCabe
> > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> > > individual
> > > > > > > > > partitions to
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > > > these
> > > > > > > > operations
> > > > > > > > > to
> > > > > > > > > > > be at
> > > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > > > good
> > > > > > > > improvements
> > > > > > > > > of
> > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > > > the
> > > > > > > > Cancellation
> > > > > > > > > of
> > > > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > > > when
> > > the
> > > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > > > empty
> > > > > replicas
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > > > implementing
> > > the
> > > > > the
> > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> > > it
> > > > > will
> > > > > > > > make
> > > > > > > > > > > KIP-236
> > > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > > > all
> > > > > pending
> > > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > > one just needs to do a
> > > > > ListPartitionRessignmentRequest,
> > > > > > > > > then
> > > > > > > > > > > submit
> > > > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > > > topic/partitions
> > > in
> > > > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > > > PDT,
> > > Colin
> > > > > > > McCabe
> > > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > > > McCabe
> > > > > wrote:
> > > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> > > Gustafson
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > > > the
> > > > > partition
> > > > > > > > > level
> > > > > > > > > > > is
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > > > understand
> > > > > > > that
> > > > > > > > > > > there is
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > reassignment in progress. This makes
> > > KIP-352 a
> > > > > > > > trivial
> > > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > > > whether
> > > the
> > > > > > > leader
> > > > > > > > > and
> > > > > > > > > > > isr
> > > > > > > > > > > > > znode
> > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > right place to store the target
> > > reassignment.
> > > > > It
> > > > > > > is a
> > > > > > > > > bit
> > > > > > > > > > > odd
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > > > from
> > > the
> > > > > > > > current
> > > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > > assume the thinking is probably that
> > > although
> > > > > the
> > > > > > > > > current
> > > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > > > znode as
> > > > > well, it
> > > > > > > > is
> > > > > > > > > > > hard to
> > > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> > > But
> > > > > if we
> > > > > > > > > have no
> > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > > the assignment znode, do you see a
> > > downside to
> > > > > > > > storing
> > > > > > > > > the
> > > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > > > better
> > > to
> > > > > keep
> > > > > > > the
> > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > > > current
> > > > > > > assignment,
> > > > > > > > > for
> > > > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > > > we
> > > should
> > > > > > > stick
> > > > > > > > > with
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > > > reassignment
> > > state
> > > > > into
> > > > > > > > > > > > > > > > >
> > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > > The
> > > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > > > epoch
> > > for the
> > > > > > > > > partition
> > > > > > > > > > > when
> > > > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > > > leader
> > > > > epoch
> > > > > > > > > resides
> > > > > > > > > > > in
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > znode anyway.  I agree there is some
> > > inconsistency
> > > > > > > here,
> > > > > > > > > but
> > > > > > > > > > > so be
> > > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > > > data
> > > > > > > structures,
> > > > > > > > > we
> > > > > > > > > > > might
> > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > > > will
> > > work
> > > > > fine
> > > > > > > > and
> > > > > > > > > be
> > > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > > > altering,
> > > > > right?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > > > reads
> > > a
> > > > > little
> > > > > > > > > > > better, and
> > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > > > +the
> > > wiki.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > > > Metadata
> > > API? In
> > > > > > > other
> > > > > > > > > > > words,
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> > > probably
> > > > > > > need
> > > > > > > > a
> > > > > > > > > > > change
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> > > alternative I
> > > > > can
> > > > > > > > > think of
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > > > Metadata
> > > > > request
> > > > > > > as
> > > > > > > > > the
> > > > > > > > > > > union
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > > > can't
> > > think
> > > > > of
> > > > > > > any
> > > > > > > > > > > benefit
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > > > this, we
> > > > > > > probably
> > > > > > > > > > > wouldn't
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > > > think on
> > > > > balance,
> > > > > > > > > you're
> > > > > > > > > > > > > right.
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > > > with
> > > the
> > > > > > > replica
> > > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > > > information is
> > > > > > > > > > > available in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > > > this,
> > > we'll
> > > > > need
> > > > > > > a
> > > > > > > > > flag
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > > > show me
> > > > > > > > reassigning
> > > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > > > stick
> > > with
> > > > > the
> > > > > > > > > original
> > > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > > ListPartitionReassignments
> > > > > > > > > API.
> > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > > > or
> > > > > Consumer
> > > > > > > > needs
> > > > > > > > > to
> > > > > > > > > > > know
> > > > > > > > > > > > > > > > > about a reassignment.  They should just be
> > > notified
> > > > > > > when
> > > > > > > > > the
> > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > partitions changes, which doesn't require
> > > changes
> > > > > to
> > > > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > > > client
> > > only
> > > > > cares
> > > > > > > if
> > > > > > > > > > > someone
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > managing the reassignment.  So adding this
> > > state
> > > > > to the
> > > > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > > > benefit.  In
> > > > > > > > the
> > > > > > > > > > > common
> > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> > > would
> > > > > be 4
> > > > > > > > > bytes per
> > > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > > MetadataResponse.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > In general, I think we have a problem of
> > > > > oversharing in
> > > > > > > > the
> > > > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > > > 100x
> > > the
> > > > > number
> > > > > > > > of
> > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > > > about
> > > giving
> > > > > > > > clients
> > > > > > > > > > > only
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > information they actually need, about the
> > > > > partitions
> > > > > > > they
> > > > > > > > > > > actually
> > > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > > > in the
> > > > > > > category
> > > > > > > > of
> > > > > > > > > > > state
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > isn't needed by clients (except very
> > > specialized
> > > > > > > > > rebalancing
> > > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > > > that
> > > > > someone
> > > > > > > > > managing
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > > > up-to-date
> > > > > > > > information,
> > > > > > > > > > > which
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > > > adding
> > > this
> > > > > > > state
> > > > > > > > to
> > > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > > or describeTopics, which could contact any
> > > node in
> > > > > the
> > > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > > > describeTopics
> > > > > > > > feels
> > > > > > > > > > > like a
> > > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > > > interacts
> > > with
> > > > > other
> > > > > > > > > extra
> > > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > > > I
> > > think
> > > > > a
> > > > > > > > > separate
> > > > > > > > > > > API
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > > > will
> > > join
> > > > > the
> > > > > > > > ISR.
> > > > > > > > > > > Will we
> > > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > > target replicas joining the ISR before
> > > taking
> > > > > the
> > > > > > > > > replica
> > > > > > > > > > > out
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > > > target
> > > > > replicas
> > > > > > > can
> > > > > > > > > > > still be
> > > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > We'll take a replica out of the target
> > > replicas
> > > > > set
> > > > > > > as
> > > > > > > > > soon
> > > > > > > > > > > as
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > > > this
> > > in
> > > > > the
> > > > > > > KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > > > permissions
> > > for
> > > > > the
> > > > > > > new
> > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Good point.  I think
> > > alterPartitionAssignments
> > > > > should
> > > > > > > > > require
> > > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > > > will be
> > > > > > > > unchanged.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> > > Shapira <
> > > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > > > have
> > > > > this new
> > > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > > > Colin
> > > McCabe
> > > > > <
> > > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > > > Create
> > > > > > > an
> > > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > > > this
> > > KIP is
> > > > > > > > > important
> > > > > > > > > > > since
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > > reassignment
> > > > > > > > > (see
> > > > > > > > > > > the
> > > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > > > discussions
> > > > > > > > we've
> > > > > > > > > had
> > > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > > > the
> > > > > important
> > > > > > > > > KIP-4
> > > > > > > > > > > goal
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > > > discussion in
> > > > > > > > the
> > > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > explicitly
> > > > > > > > > > > cancel a
> > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > > > a
> > > > > different
> > > > > > > > > > > reassignment
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > > > simplified
> > > the
> > > > > API
> > > > > > > a
> > > > > > > > > bit by
> > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > > > is
> > > used
> > > > > by
> > > > > > > both
> > > > > > > > > the
> > > > > > > > > > > alter
> > > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > > > now
> > > > > > > deprecate
> > > > > > > > > the
> > > > > > > > > > > old
> > > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > > rather than removing it completely.
> > > That
> > > > > > > should
> > > > > > > > > give
> > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > > > the
> > > new
> > > > > API.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > > > asked, I
> > > > > added a
> > > > > > > > note
> > > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > > > now
> > > use a
> > > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > > https://nam06.safelinks.protection.outlook.com/?url=https%3A
> > > > > > > > %2F%2Ftwitter.com%2FConfluentInc&data=02%7C01%7Ckchitta%
> > > > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&s
> > > > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&re
> > > > > > > > served=0>
> > > > > > > > > |
> > > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.
> > > > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&data=02%7C01%7
> > > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > > 003622876341136&sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > > &reserved=0>
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > --
> > > > > > > > > > > Gwen Shapira
> > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi Koushik,

Thanks for the idea.  This KIP is already pretty big, so I think we'll have to consider ideas like this in follow-on KIPs.

In general, figuring out what's wrong with replication is a pretty tough problem.  If we had an API for this, we'd probably want it to be unified, and not specific to reassigning partitions.

regards,
Colin


On Tue, Aug 6, 2019, at 10:57, Koushik Chitta wrote:
> Hey Colin,
> 
> Can the ListPartitionReassignmentsResult include the status of the 
> current reassignment progress of each partition? A reassignment can be 
> in progress for different reasons and the status can give the option to 
> alter the current reassignment.
> 
> Example -  A leaderISRRequest of a new assigned replicas can be 
> ignored/errored because of a storage exception.  And reassignment batch 
> will be waiting indefinitely for the new assigned replicas to be in 
> sync with the leader of the partition.  
> 	      Showing the status will give an option to alter the affected 
> partitions and allow the batch to complete reassignment.
> 
> OAR = {1, 2, 3} and RAR = {4,5,6}
> 
>      AR                 leader/isr
>     {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest 
> was lost/skipped for 5 and the reassignment operation will be waiting 
> indefinitely for the 5 to be insync.
> 
> 
> 
> Thanks,
> Koushik
> 
> -----Original Message-----
> From: Jun Rao <ju...@confluent.io> 
> Sent: Friday, August 2, 2019 10:04 AM
> To: dev <de...@kafka.apache.org>
> Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> Reassignment
> 
> Hi, Colin,
> 
> First, since we are changing the format of LeaderAndIsrRequest, which 
> is an inter broker request, it seems that we will need IBP during 
> rolling upgrade. Could we add that to the compatibility section?
> 
> Regarding UnsupportedVersionException, even without ZK node version 
> bump, we probably want to only use the new ZK value fields after all 
> brokers have been upgraded to the new binary. Otherwise, the 
> reassignment task may not be completed if the controller changes to a 
> broker still on the old binary.
> IBP is one way to achieve that. The main thing is that we need some way 
> for the controller to deal with the new ZK fields. Dealing with the 
> additional ZK node version bump seems a small thing on top of that?
> 
> Thanks,
> 
> Jun
> 
> On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > > Hi, Colin,
> > >
> > > 10. Sounds good.
> > >
> > > 13. Our current convention is to bump up the version of ZK value if 
> > > there is any format change. For example, we have bumped up the 
> > > version of the value in /brokers/ids/nnn multiple times and all of 
> > > those changes are compatible (just adding new fields). This has the 
> > > slight benefit that it makes it clear there is a format change. 
> > > Rolling upgrades and downgrades can still be supported with the 
> > > version bump. For example, if you
> > downgrade
> > > from a compatible change, you can leave the new format in ZK and the 
> > > old code will only pick up fields relevant to the old version. 
> > > Upgrade will
> > be
> > > controlled by inter broker protocol.
> >
> > Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> > protocol version.  We also need to return UnsupportedVersionException 
> > from the alterPartitionReassignments and listPartitionReassignments 
> > APIs when the IBP is too low.  This sounds doable, although we might 
> > need a release note that upgrading the IBP is necessary to allow 
> > reassignment operations after an upgrade.
> >
> > best,
> > Colin
> >
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Thanks for taking another look at this.
> > > >
> > > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > > Hi, Stan,
> > > > >
> > > > > Thanks for the explanation.
> > > > >
> > > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > > future
> > work,
> > > > > perhaps they should be added when we do the future work instead 
> > > > > of
> > now?
> > > >
> > > > I think this ties in with one of the big goals of this KIP, making 
> > > > it possible to distinguish reassigning replicas from normal replicas.
> > This is
> > > > the key to follow-on work like being able to ensure that 
> > > > partitions
> > with a
> > > > reassignment don't get falsely flagged as under-replicated in the
> > metrics,
> > > > or implementing reassignment quotas that don't accidentally affect
> > normal
> > > > replication traffic when a replica falls out of the ISR.
> > > >
> > > > For these follow-on improvements, we need to have that information 
> > > > in LeaderAndIsrRequest.  We could add the information in a 
> > > > follow-on KIP,
> > of
> > > > course, but then all the improvements are blocked on that 
> > > > follow-on
> > KIP.
> > > > That would slow things down for all of the downstream KIPs that 
> > > > are
> > blocked
> > > > on this.
> > > >
> > > > Also, to keep things consistent, I think it would be best if the
> > format of
> > > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > > I think
> > it
> > > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > > >
> > > > > > > Should we include those two fields in UpdateMetadata and
> > potentially
> > > > > > > Metadata requests too?
> > > >
> > > > We had some discussion earlier about how metadata responses to 
> > > > clients
> > are
> > > > getting too large, in part because they include a lot of 
> > > > information
> > that
> > > > most clients don't need (such as the ISR).  I think reassignment 
> > > > information definitely falls in the category of something a client
> > doesn't
> > > > need to know, so we shouldn't include it.
> > > >
> > > > A program like CruiseControl, or the command-line reassignment 
> > > > program, just wants to get the most up-to-date information about 
> > > > the state of reassigning partitions.  The MetadataRequest API 
> > > > wouldn't deliver that, because there are inherently delays in how 
> > > > we propagate metadata to brokers.  That's why the 
> > > > ListPartitionReassignments API is a better
> > choice
> > > > for those programs.  So I think if we added this information to 
> > > > the MetadataResponse, nobody would actually use it, and it would 
> > > > just use
> > up
> > > > more bandwidth.
> > > >
> > > > Of course, we can always revisit this later if we find a scenario
> > where a
> > > > producer or consumer would actually care about this.  But I think 
> > > > we
> > should
> > > > default to not adding stuff to the metadata response if we don't 
> > > > have a good use case in mind.
> > > >
> > > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > > we
> > > > cancel the
> > > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > > them
> > from
> > > > (the
> > > > > > > updated from the last-reassignment) R and TR, and starting
> > anew." In
> > > > this
> > > > > > > case, it seems that the controller needs to issue a 
> > > > > > > StopReplica
> > > > request to
> > > > > > > remove those unneeded replicas.
> > > >
> > > > Good catch.  Yes, we should document this in the  KIP.
> > > >
> > > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > > AR
> > from
> > > > R,
> > > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> > cancel
> > > > the
> > > > > > > replica movements that have not yet completed." Similar to 
> > > > > > > the
> > > > above, it
> > > > > > > seems the controller needs to issue a StopReplica request to
> > remove
> > > > those
> > > > > > > unneeded replicas.
> > > >
> > > > Right.  Let's add this.
> > > >
> > > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> > should
> > > > we bump
> > > > > > > up the version number in the json value?
> > > >
> > > > The change to the zNode is backwards compatible, though.  Older 
> > > > brokers will continue to work, but just ignore the new fields.  If 
> > > > we bump that version number, then downgrades will require hand-editing zookeeper.
> > (Of
> > > > course downgrade isn't officially supported, but it would be nice 
> > > > not
> > to
> > > > break it if we don't need to...)  Changing the version number 
> > > > would
> > also
> > > > create problems during a rolling upgrade.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > > <cmccabe@apache.org
> > >
> > > > wrote:
> > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> > Robert
> > > > > > Barrett,
> > > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > > Jason
> > > > > > Gustafson,
> > > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > > >
> > > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > > sql_consulting@yahoo.com
> > > > > > > > .invalid>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > >  +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > > George
> > > > > > > > > >
> > > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> > Shapira <
> > > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > > >
> > > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> > through
> > > > all
> > > > > > the
> > > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > > ended
> > up
> > > > with
> > > > > > a
> > > > > > > > > > great protocol.
> > > > > > > > > >
> > > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > > jason@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > > >
> > > > > > > > > > > -Jason
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > > cmccabe@apache.org
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > > reflect
> > the
> > > > fact
> > > > > > > that
> > > > > > > > > > we've
> > > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > > for 3
> > days
> > > > as
> > > > > > > > usual.
> > > > > > > > > > > >
> > > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > > Kozlovski
> > wrote:
> > > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > > >
> > > > > > > > > > > > > We have further iterated on the KIP in the
> > accompanying
> > > > > > > > discussion
> > > > > > > > > > thread
> > > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > > - we will internally use two collections to
> > represent a
> > > > > > > > reassignment
> > > > > > > > > > -
> > > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> > LeaderAndIsr has
> > > > > > been
> > > > > > > > > > updated
> > > > > > > > > > > > > accordingly
> > > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > > collection,
> > > > > > > > but
> > > > > > > > > > the
> > > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > > collections
> > - the
> > > > > > full
> > > > > > > > > > replica
> > > > > > > > > > > > set,
> > > > > > > > > > > > > the replicas we are adding as part of this
> > reassignment
> > > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > > proper
> > > > rollback
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > > API
> > call
> > > > > > > > > > > > >
> > > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwik
> > i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> > an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&amp;data=02%7
> > C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> > 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&amp;sdata=w
> > GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&amp;reserved=0
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Stanislav
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > > cmccabe@apache.org
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > > getting
> > a PR
> > > > > > done
> > > > > > > > as a
> > > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > > we'll
> > > > finish up
> > > > > > > the
> > > > > > > > > > vote.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > > (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > What is the estimated timeline to have this
> > > > implemented?
> > > > > > > If
> > > > > > > > any
> > > > > > > > > > help
> > > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > > cancelling
> > > > > > > > reassignments,  I
> > > > > > > > > > can
> > > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > > PDT,
> > Colin
> > > > > > McCabe
> > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> > individual
> > > > > > > > partitions to
> > > > > > > > > > be
> > > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > > these
> > > > > > > operations
> > > > > > > > to
> > > > > > > > > > be at
> > > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > > good
> > > > > > > improvements
> > > > > > > > of
> > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > > the
> > > > > > > Cancellation
> > > > > > > > of
> > > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > > when
> > the
> > > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > > empty
> > > > replicas
> > > > > > for
> > > > > > > > the
> > > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > > implementing
> > the
> > > > the
> > > > > > > > > > partition
> > > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> > it
> > > > will
> > > > > > > make
> > > > > > > > > > KIP-236
> > > > > > > > > > > > (it
> > > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > > all
> > > > pending
> > > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > > one just needs to do a
> > > > ListPartitionRessignmentRequest,
> > > > > > > > then
> > > > > > > > > > submit
> > > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > > topic/partitions
> > in
> > > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > > PDT,
> > Colin
> > > > > > McCabe
> > > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > > McCabe
> > > > wrote:
> > > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> > Gustafson
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > > the
> > > > partition
> > > > > > > > level
> > > > > > > > > > is
> > > > > > > > > > > > the
> > > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > > understand
> > > > > > that
> > > > > > > > > > there is
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > reassignment in progress. This makes
> > KIP-352 a
> > > > > > > trivial
> > > > > > > > > > > > follow-up
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > > whether
> > the
> > > > > > leader
> > > > > > > > and
> > > > > > > > > > isr
> > > > > > > > > > > > znode
> > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > right place to store the target
> > reassignment.
> > > > It
> > > > > > is a
> > > > > > > > bit
> > > > > > > > > > odd
> > > > > > > > > > > > to
> > > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > > from
> > the
> > > > > > > current
> > > > > > > > > > > > assignment,
> > > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > > assume the thinking is probably that
> > although
> > > > the
> > > > > > > > current
> > > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > > znode as
> > > > well, it
> > > > > > > is
> > > > > > > > > > hard to
> > > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> > But
> > > > if we
> > > > > > > > have no
> > > > > > > > > > > > plan
> > > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > > the assignment znode, do you see a
> > downside to
> > > > > > > storing
> > > > > > > > the
> > > > > > > > > > > > target
> > > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > > better
> > to
> > > > keep
> > > > > > the
> > > > > > > > > > target
> > > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > > current
> > > > > > assignment,
> > > > > > > > for
> > > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > > we
> > should
> > > > > > stick
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > > reassignment
> > state
> > > > into
> > > > > > > > > > > > > > > >
> > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > > The
> > > > > > > > > > > > reason is
> > > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > > epoch
> > for the
> > > > > > > > partition
> > > > > > > > > > when
> > > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > > leader
> > > > epoch
> > > > > > > > resides
> > > > > > > > > > in
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > znode anyway.  I agree there is some
> > inconsistency
> > > > > > here,
> > > > > > > > but
> > > > > > > > > > so be
> > > > > > > > > > > > it:
> > > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > > data
> > > > > > structures,
> > > > > > > > we
> > > > > > > > > > might
> > > > > > > > > > > > do
> > > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > > will
> > work
> > > > fine
> > > > > > > and
> > > > > > > > be
> > > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > > altering,
> > > > right?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > > reads
> > a
> > > > little
> > > > > > > > > > better, and
> > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > > +the
> > wiki.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > > Metadata
> > API? In
> > > > > > other
> > > > > > > > > > words,
> > > > > > > > > > > > are
> > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> > probably
> > > > > > need
> > > > > > > a
> > > > > > > > > > change
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> > alternative I
> > > > can
> > > > > > > > think of
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > > Metadata
> > > > request
> > > > > > as
> > > > > > > > the
> > > > > > > > > > union
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > > can't
> > think
> > > > of
> > > > > > any
> > > > > > > > > > benefit
> > > > > > > > > > > > to
> > > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > > this, we
> > > > > > probably
> > > > > > > > > > wouldn't
> > > > > > > > > > > > need
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > > think on
> > > > balance,
> > > > > > > > you're
> > > > > > > > > > > > right.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > > with
> > the
> > > > > > replica
> > > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > > information is
> > > > > > > > > > available in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > > this,
> > we'll
> > > > need
> > > > > > a
> > > > > > > > flag
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > > show me
> > > > > > > reassigning
> > > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > > stick
> > with
> > > > the
> > > > > > > > original
> > > > > > > > > > > > > > > > proposal of having a separate
> > > > > > ListPartitionReassignments
> > > > > > > > API.
> > > > > > > > > > > > There
> > > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > > or
> > > > Consumer
> > > > > > > needs
> > > > > > > > to
> > > > > > > > > > know
> > > > > > > > > > > > > > > > about a reassignment.  They should just be
> > notified
> > > > > > when
> > > > > > > > the
> > > > > > > > > > set of
> > > > > > > > > > > > > > > > partitions changes, which doesn't require
> > changes
> > > > to
> > > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > > client
> > only
> > > > cares
> > > > > > if
> > > > > > > > > > someone
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > managing the reassignment.  So adding this
> > state
> > > > to the
> > > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > > benefit.  In
> > > > > > > the
> > > > > > > > > > common
> > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> > would
> > > > be 4
> > > > > > > > bytes per
> > > > > > > > > > > > > > > > partition of extra overhead in the
> > > > MetadataResponse.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > In general, I think we have a problem of
> > > > oversharing in
> > > > > > > the
> > > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > > 100x
> > the
> > > > number
> > > > > > > of
> > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > > about
> > giving
> > > > > > > clients
> > > > > > > > > > only
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > information they actually need, about the
> > > > partitions
> > > > > > they
> > > > > > > > > > actually
> > > > > > > > > > > > > > care
> > > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > > in the
> > > > > > category
> > > > > > > of
> > > > > > > > > > state
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > isn't needed by clients (except very
> > specialized
> > > > > > > > rebalancing
> > > > > > > > > > > > programs).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > > that
> > > > someone
> > > > > > > > managing
> > > > > > > > > > an
> > > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > > up-to-date
> > > > > > > information,
> > > > > > > > > > which
> > > > > > > > > > > > is
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > > adding
> > this
> > > > > > state
> > > > > > > to
> > > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > > or describeTopics, which could contact any
> > node in
> > > > the
> > > > > > > > > > cluster, is
> > > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > > describeTopics
> > > > > > > feels
> > > > > > > > > > like a
> > > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > > interacts
> > with
> > > > other
> > > > > > > > extra
> > > > > > > > > > > > booleans
> > > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > > I
> > think
> > > > a
> > > > > > > > separate
> > > > > > > > > > API
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > > will
> > join
> > > > the
> > > > > > > ISR.
> > > > > > > > > > Will we
> > > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > > target replicas joining the ISR before
> > taking
> > > > the
> > > > > > > > replica
> > > > > > > > > > out
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > > target
> > > > replicas
> > > > > > can
> > > > > > > > > > still be
> > > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > We'll take a replica out of the target
> > replicas
> > > > set
> > > > > > as
> > > > > > > > soon
> > > > > > > > > > as
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > > this
> > in
> > > > the
> > > > > > KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > > permissions
> > for
> > > > the
> > > > > > new
> > > > > > > > APIs.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Good point.  I think
> > alterPartitionAssignments
> > > > should
> > > > > > > > require
> > > > > > > > > > > > ALTER
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > > will be
> > > > > > > unchanged.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> > Shapira <
> > > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > > have
> > > > this new
> > > > > > > > > > capability.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > > Colin
> > McCabe
> > > > <
> > > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > > Create
> > > > > > an
> > > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > > this
> > KIP is
> > > > > > > > important
> > > > > > > > > > since
> > > > > > > > > > > > it
> > > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > > Kafka
> > > > > > reassignment
> > > > > > > > (see
> > > > > > > > > > the
> > > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > > discussions
> > > > > > > we've
> > > > > > > > had
> > > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > > the
> > > > important
> > > > > > > > KIP-4
> > > > > > > > > > goal
> > > > > > > > > > > > of
> > > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > > discussion in
> > > > > > > the
> > > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > > need to
> > > > > > > explicitly
> > > > > > > > > > cancel a
> > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > > a
> > > > different
> > > > > > > > > > reassignment
> > > > > > > > > > > > for
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > > simplified
> > the
> > > > API
> > > > > > a
> > > > > > > > bit by
> > > > > > > > > > > > adding
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > > is
> > used
> > > > by
> > > > > > both
> > > > > > > > the
> > > > > > > > > > alter
> > > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > > now
> > > > > > deprecate
> > > > > > > > the
> > > > > > > > > > old
> > > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > > rather than removing it completely.
> > That
> > > > > > should
> > > > > > > > give
> > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > > the
> > new
> > > > API.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > > asked, I
> > > > added a
> > > > > > > note
> > > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > > now
> > use a
> > > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > > Twitter <
> > > > > > > https://nam06.safelinks.protection.outlook.com/?url=https%3A
> > > > > > > %2F%2Ftwitter.com%2FConfluentInc&amp;data=02%7C01%7Ckchitta%
> > > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&amp;s
> > > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&amp;re
> > > > > > > served=0>
> > > > > > > > |
> > > > > > > > > > blog
> > > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.
> > > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&amp;data=02%7C01%7
> > > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > > 003622876341136&amp;sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > > &amp;reserved=0>
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > --
> > > > > > > > > > Gwen Shapira
> > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

RE: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Koushik Chitta <kc...@microsoft.com.INVALID>.
Hey Colin,

Can the ListPartitionReassignmentsResult include the status of the current reassignment progress of each partition? A reassignment can be in progress for different reasons and the status can give the option to alter the current reassignment.

Example -  A leaderISRRequest of a new assigned replicas can be ignored/errored because of a storage exception.  And reassignment batch will be waiting indefinitely for the new assigned replicas to be in sync with the leader of the partition.  
	      Showing the status will give an option to alter the affected partitions and allow the batch to complete reassignment.

OAR = {1, 2, 3} and RAR = {4,5,6}

     AR                 leader/isr
    {1,2,3,4,5,6}            1/{1,2,3,4,6}       =>  LeaderISRRequest was lost/skipped for 5 and the reassignment operation will be waiting indefinitely for the 5 to be insync.



Thanks,
Koushik

-----Original Message-----
From: Jun Rao <ju...@confluent.io> 
Sent: Friday, August 2, 2019 10:04 AM
To: dev <de...@kafka.apache.org>
Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Hi, Colin,

First, since we are changing the format of LeaderAndIsrRequest, which is an inter broker request, it seems that we will need IBP during rolling upgrade. Could we add that to the compatibility section?

Regarding UnsupportedVersionException, even without ZK node version bump, we probably want to only use the new ZK value fields after all brokers have been upgraded to the new binary. Otherwise, the reassignment task may not be completed if the controller changes to a broker still on the old binary.
IBP is one way to achieve that. The main thing is that we need some way for the controller to deal with the new ZK fields. Dealing with the additional ZK node version bump seems a small thing on top of that?

Thanks,

Jun

On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:

> On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > Hi, Colin,
> >
> > 10. Sounds good.
> >
> > 13. Our current convention is to bump up the version of ZK value if 
> > there is any format change. For example, we have bumped up the 
> > version of the value in /brokers/ids/nnn multiple times and all of 
> > those changes are compatible (just adding new fields). This has the 
> > slight benefit that it makes it clear there is a format change. 
> > Rolling upgrades and downgrades can still be supported with the 
> > version bump. For example, if you
> downgrade
> > from a compatible change, you can leave the new format in ZK and the 
> > old code will only pick up fields relevant to the old version. 
> > Upgrade will
> be
> > controlled by inter broker protocol.
>
> Hmm.  If we bump that ZK node version, we will need a new inter-broker 
> protocol version.  We also need to return UnsupportedVersionException 
> from the alterPartitionReassignments and listPartitionReassignments 
> APIs when the IBP is too low.  This sounds doable, although we might 
> need a release note that upgrading the IBP is necessary to allow 
> reassignment operations after an upgrade.
>
> best,
> Colin
>
> >
> > Thanks,
> >
> > Jun
> >
> > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi Jun,
> > >
> > > Thanks for taking another look at this.
> > >
> > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > Hi, Stan,
> > > >
> > > > Thanks for the explanation.
> > > >
> > > > 10. If those new fields in LeaderAndIsr are only needed for 
> > > > future
> work,
> > > > perhaps they should be added when we do the future work instead 
> > > > of
> now?
> > >
> > > I think this ties in with one of the big goals of this KIP, making 
> > > it possible to distinguish reassigning replicas from normal replicas.
> This is
> > > the key to follow-on work like being able to ensure that 
> > > partitions
> with a
> > > reassignment don't get falsely flagged as under-replicated in the
> metrics,
> > > or implementing reassignment quotas that don't accidentally affect
> normal
> > > replication traffic when a replica falls out of the ISR.
> > >
> > > For these follow-on improvements, we need to have that information 
> > > in LeaderAndIsrRequest.  We could add the information in a 
> > > follow-on KIP,
> of
> > > course, but then all the improvements are blocked on that 
> > > follow-on
> KIP.
> > > That would slow things down for all of the downstream KIPs that 
> > > are
> blocked
> > > on this.
> > >
> > > Also, to keep things consistent, I think it would be best if the
> format of
> > > the data in the LeaderAndIsrRequest matched the format of the data 
> > > in ZooKeeper.  Since we're deciding on the ZK format in this KIP, 
> > > I think
> it
> > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > >
> > > > > > Should we include those two fields in UpdateMetadata and
> potentially
> > > > > > Metadata requests too?
> > >
> > > We had some discussion earlier about how metadata responses to 
> > > clients
> are
> > > getting too large, in part because they include a lot of 
> > > information
> that
> > > most clients don't need (such as the ISR).  I think reassignment 
> > > information definitely falls in the category of something a client
> doesn't
> > > need to know, so we shouldn't include it.
> > >
> > > A program like CruiseControl, or the command-line reassignment 
> > > program, just wants to get the most up-to-date information about 
> > > the state of reassigning partitions.  The MetadataRequest API 
> > > wouldn't deliver that, because there are inherently delays in how 
> > > we propagate metadata to brokers.  That's why the 
> > > ListPartitionReassignments API is a better
> choice
> > > for those programs.  So I think if we added this information to 
> > > the MetadataResponse, nobody would actually use it, and it would 
> > > just use
> up
> > > more bandwidth.
> > >
> > > Of course, we can always revisit this later if we find a scenario
> where a
> > > producer or consumer would actually care about this.  But I think 
> > > we
> should
> > > default to not adding stuff to the metadata response if we don't 
> > > have a good use case in mind.
> > >
> > > > > > 11. "If a new reassignment is issued during an on-going one, 
> > > > > > we
> > > cancel the
> > > > > > current one by emptying out both AR and RR, constructing 
> > > > > > them
> from
> > > (the
> > > > > > updated from the last-reassignment) R and TR, and starting
> anew." In
> > > this
> > > > > > case, it seems that the controller needs to issue a 
> > > > > > StopReplica
> > > request to
> > > > > > remove those unneeded replicas.
> > >
> > > Good catch.  Yes, we should document this in the  KIP.
> > >
> > > > > > 12. "Essentially, once a cancellation is called we subtract 
> > > > > > AR
> from
> > > R,
> > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> cancel
> > > the
> > > > > > replica movements that have not yet completed." Similar to 
> > > > > > the
> > > above, it
> > > > > > seems the controller needs to issue a StopReplica request to
> remove
> > > those
> > > > > > unneeded replicas.
> > >
> > > Right.  Let's add this.
> > >
> > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> should
> > > we bump
> > > > > > up the version number in the json value?
> > >
> > > The change to the zNode is backwards compatible, though.  Older 
> > > brokers will continue to work, but just ignore the new fields.  If 
> > > we bump that version number, then downgrades will require hand-editing zookeeper.
> (Of
> > > course downgrade isn't officially supported, but it would be nice 
> > > not
> to
> > > break it if we don't need to...)  Changing the version number 
> > > would
> also
> > > create problems during a rolling upgrade.
> > >
> > > best,
> > > Colin
> > >
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe 
> > > > > > <cmccabe@apache.org
> >
> > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> Robert
> > > > > Barrett,
> > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, 
> > > > > > > Jason
> > > > > Gustafson,
> > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > >
> > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > sql_consulting@yahoo.com
> > > > > > > .invalid>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > >  +1 (non-binding)
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > George
> > > > > > > > >
> > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> Shapira <
> > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > >
> > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> through
> > > all
> > > > > the
> > > > > > > > > questions, edge cases, requests and alternatives. We 
> > > > > > > > > ended
> up
> > > with
> > > > > a
> > > > > > > > > great protocol.
> > > > > > > > >
> > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > >
> > > > > > > > > > -Jason
> > > > > > > > > >
> > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > cmccabe@apache.org
> > > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to 
> > > > > > > > > > > reflect
> the
> > > fact
> > > > > > that
> > > > > > > > > we've
> > > > > > > > > > > made significant changes.  The new vote will go 
> > > > > > > > > > > for 3
> days
> > > as
> > > > > > > usual.
> > > > > > > > > > >
> > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav 
> > > > > > > > > > > Kozlovski
> wrote:
> > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > >
> > > > > > > > > > > > We have further iterated on the KIP in the
> accompanying
> > > > > > > discussion
> > > > > > > > > thread
> > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > >
> > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > - we will internally use two collections to
> represent a
> > > > > > > reassignment
> > > > > > > > > -
> > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> LeaderAndIsr has
> > > > > been
> > > > > > > > > updated
> > > > > > > > > > > > accordingly
> > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > collection,
> > > > > > > but
> > > > > > > > > the
> > > > > > > > > > > > List API will now return three separate 
> > > > > > > > > > > > collections
> - the
> > > > > full
> > > > > > > > > replica
> > > > > > > > > > > set,
> > > > > > > > > > > > the replicas we are adding as part of this
> reassignment
> > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > and the replicas we are removing 
> > > > > > > > > > > > ("removingReplicas")
> > > > > > > > > > > > - cancellation of a reassignment now means a 
> > > > > > > > > > > > proper
> > > rollback
> > > > > of
> > > > > > > the
> > > > > > > > > > > > assignment to its original state prior to the 
> > > > > > > > > > > > API
> call
> > > > > > > > > > > >
> > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwik
> i.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-455%253A%2BCreate%2B
> an%2BAdministrative%2BAPI%2Bfor%2BReplica%2BReassignment&amp;data=02%7
> C01%7Ckchitta%40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f
> 988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876331137&amp;sdata=w
> GYsHbEN5ApACnH5XOswidEnaUjjLOdmbWD0B3Z3xCs%3D&amp;reserved=0
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Stanislav
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > cmccabe@apache.org
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for taking a look.  I am working on 
> > > > > > > > > > > > > getting
> a PR
> > > > > done
> > > > > > > as a
> > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then 
> > > > > > > > > > > > > we'll
> > > finish up
> > > > > > the
> > > > > > > > > vote.
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > What is the estimated timeline to have this
> > > implemented?
> > > > > > If
> > > > > > > any
> > > > > > > > > help
> > > > > > > > > > > > > > is needed in the implementation of 
> > > > > > > > > > > > > > cancelling
> > > > > > > reassignments,  I
> > > > > > > > > can
> > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > George
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM 
> > > > > > > > > > > > > > PDT,
> Colin
> > > > > McCabe
> > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> individual
> > > > > > > partitions to
> > > > > > > > > be
> > > > > > > > > > > > > > cancelled.  I think it's very important for 
> > > > > > > > > > > > > > these
> > > > > > operations
> > > > > > > to
> > > > > > > > > be at
> > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very 
> > > > > > > > > > > > > > > good
> > > > > > improvements
> > > > > > > of
> > > > > > > > > Kafka
> > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > One question, looks like the KIP includes 
> > > > > > > > > > > > > > > the
> > > > > > Cancellation
> > > > > > > of
> > > > > > > > > > > > > > > individual pending reassignments as well 
> > > > > > > > > > > > > > > when
> the
> > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has 
> > > > > > > > > > > > > > > empty
> > > replicas
> > > > > for
> > > > > > > the
> > > > > > > > > > > > > > > topic/partition. Will you also be 
> > > > > > > > > > > > > > > implementing
> the
> > > the
> > > > > > > > > partition
> > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> it
> > > will
> > > > > > make
> > > > > > > > > KIP-236
> > > > > > > > > > > (it
> > > > > > > > > > > > > > > has PR already) trivial, since the cancel 
> > > > > > > > > > > > > > > all
> > > pending
> > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > one just needs to do a
> > > ListPartitionRessignmentRequest,
> > > > > > > then
> > > > > > > > > submit
> > > > > > > > > > > > > > > empty replicas for all those 
> > > > > > > > > > > > > > > topic/partitions
> in
> > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM 
> > > > > > > > > > > > > > > PDT,
> Colin
> > > > > McCabe
> > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin 
> > > > > > > > > > > > > > > McCabe
> > > wrote:
> > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> Gustafson
> > > > > wrote:
> > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I think storing reassignment state at 
> > > > > > > > > > > > > > > > > the
> > > partition
> > > > > > > level
> > > > > > > > > is
> > > > > > > > > > > the
> > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > understand
> > > > > that
> > > > > > > > > there is
> > > > > > > > > > > a
> > > > > > > > > > > > > > > > > reassignment in progress. This makes
> KIP-352 a
> > > > > > trivial
> > > > > > > > > > > follow-up
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > example. The only doubt I have is 
> > > > > > > > > > > > > > > > > whether
> the
> > > > > leader
> > > > > > > and
> > > > > > > > > isr
> > > > > > > > > > > znode
> > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > right place to store the target
> reassignment.
> > > It
> > > > > is a
> > > > > > > bit
> > > > > > > > > odd
> > > > > > > > > > > to
> > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > target assignment in a separate place 
> > > > > > > > > > > > > > > > > from
> the
> > > > > > current
> > > > > > > > > > > assignment,
> > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > assume the thinking is probably that
> although
> > > the
> > > > > > > current
> > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > probably be in the leader and isr 
> > > > > > > > > > > > > > > > > znode as
> > > well, it
> > > > > > is
> > > > > > > > > hard to
> > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> But
> > > if we
> > > > > > > have no
> > > > > > > > > > > plan
> > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > the assignment znode, do you see a
> downside to
> > > > > > storing
> > > > > > > the
> > > > > > > > > > > target
> > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > That's a good point -- it's probably 
> > > > > > > > > > > > > > > > better
> to
> > > keep
> > > > > the
> > > > > > > > > target
> > > > > > > > > > > > > > > > assignment in the same znode as the 
> > > > > > > > > > > > > > > > current
> > > > > assignment,
> > > > > > > for
> > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I took another look at this, and I think 
> > > > > > > > > > > > > > > we
> should
> > > > > stick
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > > > > initial proposal of putting the 
> > > > > > > > > > > > > > > reassignment
> state
> > > into
> > > > > > > > > > > > > > >
> > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > The
> > > > > > > > > > > reason is
> > > > > > > > > > > > > > > because we'll want to bump the leader 
> > > > > > > > > > > > > > > epoch
> for the
> > > > > > > partition
> > > > > > > > > when
> > > > > > > > > > > > > > > changing the reassignment state, and the 
> > > > > > > > > > > > > > > leader
> > > epoch
> > > > > > > resides
> > > > > > > > > in
> > > > > > > > > > > that
> > > > > > > > > > > > > > > znode anyway.  I agree there is some
> inconsistency
> > > > > here,
> > > > > > > but
> > > > > > > > > so be
> > > > > > > > > > > it:
> > > > > > > > > > > > > > > if we were to greenfield these zookeeper 
> > > > > > > > > > > > > > > data
> > > > > structures,
> > > > > > > we
> > > > > > > > > might
> > > > > > > > > > > do
> > > > > > > > > > > > > > > it differently, but the proposed scheme 
> > > > > > > > > > > > > > > will
> work
> > > fine
> > > > > > and
> > > > > > > be
> > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1. Should 
> > > > > > > > > > > > > > > > > `alterPartitionReassignments` be
> > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > It's the current assignment we're 
> > > > > > > > > > > > > > > > > altering,
> > > right?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments 
> > > > > > > > > > > > > > > > reads
> a
> > > little
> > > > > > > > > better, and
> > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in 
> > > > > > > > > > > > > > > +the
> wiki.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2. Does this change affect the 
> > > > > > > > > > > > > > > > > Metadata
> API? In
> > > > > other
> > > > > > > > > words,
> > > > > > > > > > > are
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> probably
> > > > > need
> > > > > > a
> > > > > > > > > change
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> alternative I
> > > can
> > > > > > > think of
> > > > > > > > > > > would
> > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > represent the replica set in the 
> > > > > > > > > > > > > > > > > Metadata
> > > request
> > > > > as
> > > > > > > the
> > > > > > > > > union
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > current and target replicas, but I 
> > > > > > > > > > > > > > > > > can't
> think
> > > of
> > > > > any
> > > > > > > > > benefit
> > > > > > > > > > > to
> > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > reassignments. Note that if we did 
> > > > > > > > > > > > > > > > > this, we
> > > > > probably
> > > > > > > > > wouldn't
> > > > > > > > > > > need
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I thought about this a bit... and I 
> > > > > > > > > > > > > > > > think on
> > > balance,
> > > > > > > you're
> > > > > > > > > > > right.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > should keep this information together 
> > > > > > > > > > > > > > > > with
> the
> > > > > replica
> > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > information is
> > > > > > > > > available in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > >  However, I do think in order to do 
> > > > > > > > > > > > > > > > this,
> we'll
> > > need
> > > > > a
> > > > > > > flag
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > MetadataRequest that specifiies "only 
> > > > > > > > > > > > > > > > show me
> > > > > > reassigning
> > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I revisited this, and I think we should 
> > > > > > > > > > > > > > > stick
> with
> > > the
> > > > > > > original
> > > > > > > > > > > > > > > proposal of having a separate
> > > > > ListPartitionReassignments
> > > > > > > API.
> > > > > > > > > > > There
> > > > > > > > > > > > > > > really is no use case where the Producer 
> > > > > > > > > > > > > > > or
> > > Consumer
> > > > > > needs
> > > > > > > to
> > > > > > > > > know
> > > > > > > > > > > > > > > about a reassignment.  They should just be
> notified
> > > > > when
> > > > > > > the
> > > > > > > > > set of
> > > > > > > > > > > > > > > partitions changes, which doesn't require
> changes
> > > to
> > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin 
> > > > > > > > > > > > > > > client
> only
> > > cares
> > > > > if
> > > > > > > > > someone
> > > > > > > > > > > is
> > > > > > > > > > > > > > > managing the reassignment.  So adding this
> state
> > > to the
> > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > benefit.  In
> > > > > > the
> > > > > > > > > common
> > > > > > > > > > > > > case
> > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> would
> > > be 4
> > > > > > > bytes per
> > > > > > > > > > > > > > > partition of extra overhead in the
> > > MetadataResponse.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > In general, I think we have a problem of
> > > oversharing in
> > > > > > the
> > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 
> > > > > > > > > > > > > > > 100x
> the
> > > number
> > > > > > of
> > > > > > > > > > > partitions
> > > > > > > > > > > > > > > we support, we'll need to get stricter 
> > > > > > > > > > > > > > > about
> giving
> > > > > > clients
> > > > > > > > > only
> > > > > > > > > > > the
> > > > > > > > > > > > > > > information they actually need, about the
> > > partitions
> > > > > they
> > > > > > > > > actually
> > > > > > > > > > > > > care
> > > > > > > > > > > > > > > about.  Reassignment state clearly falls 
> > > > > > > > > > > > > > > in the
> > > > > category
> > > > > > of
> > > > > > > > > state
> > > > > > > > > > > that
> > > > > > > > > > > > > > > isn't needed by clients (except very
> specialized
> > > > > > > rebalancing
> > > > > > > > > > > programs).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Another important consideration here is 
> > > > > > > > > > > > > > > that
> > > someone
> > > > > > > managing
> > > > > > > > > an
> > > > > > > > > > > > > > > ongoing reassignment wants the most 
> > > > > > > > > > > > > > > up-to-date
> > > > > > information,
> > > > > > > > > which
> > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > be found on the controller.  Therefore 
> > > > > > > > > > > > > > > adding
> this
> > > > > state
> > > > > > to
> > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > or describeTopics, which could contact any
> node in
> > > the
> > > > > > > > > cluster, is
> > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > describeTopics
> > > > > > feels
> > > > > > > > > like a
> > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > API.  It's an extra boolean which 
> > > > > > > > > > > > > > > interacts
> with
> > > other
> > > > > > > extra
> > > > > > > > > > > booleans
> > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  
> > > > > > > > > > > > > > > I
> think
> > > a
> > > > > > > separate
> > > > > > > > > API
> > > > > > > > > > > is
> > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 3. As replicas come into sync, they 
> > > > > > > > > > > > > > > > > will
> join
> > > the
> > > > > > ISR.
> > > > > > > > > Will we
> > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > target replicas joining the ISR before
> taking
> > > the
> > > > > > > replica
> > > > > > > > > out
> > > > > > > > > > > of
> > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > replicas set? Also, I assume that 
> > > > > > > > > > > > > > > > > target
> > > replicas
> > > > > can
> > > > > > > > > still be
> > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > We'll take a replica out of the target
> replicas
> > > set
> > > > > as
> > > > > > > soon
> > > > > > > > > as
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify 
> > > > > > > > > > > > > > > > this
> in
> > > the
> > > > > KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 4. Probably useful to mention 
> > > > > > > > > > > > > > > > > permissions
> for
> > > the
> > > > > new
> > > > > > > APIs.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Good point.  I think
> alterPartitionAssignments
> > > should
> > > > > > > require
> > > > > > > > > > > ALTER
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions 
> > > > > > > > > > > > > > > > will be
> > > > > > unchanged.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks, Jason
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> Shapira <
> > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > Looks great, and will be awesome to 
> > > > > > > > > > > > > > > > > > have
> > > this new
> > > > > > > > > capability.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM 
> > > > > > > > > > > > > > > > > > Colin
> McCabe
> > > <
> > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > Create
> > > > > an
> > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think 
> > > > > > > > > > > > > > > > > > > this
> KIP is
> > > > > > > important
> > > > > > > > > since
> > > > > > > > > > > it
> > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > many follow-on improvements to 
> > > > > > > > > > > > > > > > > > > Kafka
> > > > > reassignment
> > > > > > > (see
> > > > > > > > > the
> > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > discussions
> > > > > > we've
> > > > > > > had
> > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > reassignment).  It also furthers 
> > > > > > > > > > > > > > > > > > > the
> > > important
> > > > > > > KIP-4
> > > > > > > > > goal
> > > > > > > > > > > of
> > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > discussion in
> > > > > > the
> > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > Robert suggested, I removed the 
> > > > > > > > > > > > > > > > > > > need to
> > > > > > explicitly
> > > > > > > > > cancel a
> > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > for a partition before setting up 
> > > > > > > > > > > > > > > > > > > a
> > > different
> > > > > > > > > reassignment
> > > > > > > > > > > for
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > specific partition.  I also 
> > > > > > > > > > > > > > > > > > > simplified
> the
> > > API
> > > > > a
> > > > > > > bit by
> > > > > > > > > > > adding
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > PartitionReassignment class which 
> > > > > > > > > > > > > > > > > > > is
> used
> > > by
> > > > > both
> > > > > > > the
> > > > > > > > > alter
> > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I modified the proposal so that we 
> > > > > > > > > > > > > > > > > > > now
> > > > > deprecate
> > > > > > > the
> > > > > > > > > old
> > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > rather than removing it completely.
> That
> > > > > should
> > > > > > > give
> > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > tools some time to transition to 
> > > > > > > > > > > > > > > > > > > the
> new
> > > API.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > To clarify a question Viktor 
> > > > > > > > > > > > > > > > > > > asked, I
> > > added a
> > > > > > note
> > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will 
> > > > > > > > > > > > > > > > > > > now
> use a
> > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > thanks, Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > *Gwen Shapira* Product Manager | 
> > > > > > > > > > > > > > > > > > Confluent
> > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap Follow us: 
> > > > > > > > > > > > > > > > > > Twitter <
> > > > > > https://nam06.safelinks.protection.outlook.com/?url=https%3A
> > > > > > %2F%2Ftwitter.com%2FConfluentInc&amp;data=02%7C01%7Ckchitta%
> > > > > > 40microsoft.com%7C09235110291f453cb18608d7176b8505%7C72f988b
> > > > > > f86f141af91ab2d7cd011db47%7C1%7C0%7C637003622876341136&amp;s
> > > > > > data=nk22CVhfV%2FTqNRFQc04LLldSHmFHTvrxcxYZcORpITg%3D&amp;re
> > > > > > served=0>
> > > > > > > |
> > > > > > > > > blog
> > > > > > > > > > > > > > > > > > <https://nam06.safelinks.protection.
> > > > > > > > > > > > > > > > > > outlook.com/?url=http%3A%2F%2Fwww.co
> > > > > > > > > > > > > > > > > > nfluent.io%2Fblog&amp;data=02%7C01%7
> > > > > > > > > > > > > > > > > > Ckchitta%40microsoft.com%7C092351102
> > > > > > > > > > > > > > > > > > 91f453cb18608d7176b8505%7C72f988bf86
> > > > > > > > > > > > > > > > > > f141af91ab2d7cd011db47%7C1%7C0%7C637
> > > > > > > > > > > > > > > > > > 003622876341136&amp;sdata=TwKB1ey0Do
> > > > > > > > > > > > > > > > > > pQ3gmZJ88vPHxHF4hOZEhQH8z8LqhElQc%3D
> > > > > > > > > > > > > > > > > > &amp;reserved=0>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > Gwen Shapira
> > > > > > > > > Product Manager | Confluent
> > > > > > > > > 650.450.2760 | @gwenshap Follow us: Twitter | blog
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Jun Rao <ju...@confluent.io>.
Hi, Colin,

First, since we are changing the format of LeaderAndIsrRequest, which is an
inter broker request, it seems that we will need IBP during rolling
upgrade. Could we add that to the compatibility section?

Regarding UnsupportedVersionException, even without ZK node version bump,
we probably want to only use the new ZK value fields after all brokers have
been upgraded to the new binary. Otherwise, the reassignment task may not
be completed if the controller changes to a broker still on the old binary.
IBP is one way to achieve that. The main thing is that we need some way for
the controller to deal with the new ZK fields. Dealing with the additional
ZK node version bump seems a small thing on top of that?

Thanks,

Jun

On Thu, Aug 1, 2019 at 3:05 PM Colin McCabe <cm...@apache.org> wrote:

> On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> > Hi, Colin,
> >
> > 10. Sounds good.
> >
> > 13. Our current convention is to bump up the version of ZK value if there
> > is any format change. For example, we have bumped up the version of the
> > value in /brokers/ids/nnn multiple times and all of those changes are
> > compatible (just adding new fields). This has the slight benefit that it
> > makes it clear there is a format change. Rolling upgrades and downgrades
> > can still be supported with the version bump. For example, if you
> downgrade
> > from a compatible change, you can leave the new format in ZK and the old
> > code will only pick up fields relevant to the old version. Upgrade will
> be
> > controlled by inter broker protocol.
>
> Hmm.  If we bump that ZK node version, we will need a new inter-broker
> protocol version.  We also need to return UnsupportedVersionException from
> the alterPartitionReassignments and listPartitionReassignments APIs when
> the IBP is too low.  This sounds doable, although we might need a release
> note that upgrading the IBP is necessary to allow reassignment operations
> after an upgrade.
>
> best,
> Colin
>
> >
> > Thanks,
> >
> > Jun
> >
> > On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi Jun,
> > >
> > > Thanks for taking another look at this.
> > >
> > > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > > Hi, Stan,
> > > >
> > > > Thanks for the explanation.
> > > >
> > > > 10. If those new fields in LeaderAndIsr are only needed for future
> work,
> > > > perhaps they should be added when we do the future work instead of
> now?
> > >
> > > I think this ties in with one of the big goals of this KIP, making it
> > > possible to distinguish reassigning replicas from normal replicas.
> This is
> > > the key to follow-on work like being able to ensure that partitions
> with a
> > > reassignment don't get falsely flagged as under-replicated in the
> metrics,
> > > or implementing reassignment quotas that don't accidentally affect
> normal
> > > replication traffic when a replica falls out of the ISR.
> > >
> > > For these follow-on improvements, we need to have that information in
> > > LeaderAndIsrRequest.  We could add the information in a follow-on KIP,
> of
> > > course, but then all the improvements are blocked on that follow-on
> KIP.
> > > That would slow things down for all of the downstream KIPs that are
> blocked
> > > on this.
> > >
> > > Also, to keep things consistent, I think it would be best if the
> format of
> > > the data in the LeaderAndIsrRequest matched the format of the data in
> > > ZooKeeper.  Since we're deciding on the ZK format in this KIP, I think
> it
> > > makes sense to also decide on the format in the LeaderAndIsrRequest.
> > >
> > > > > > Should we include those two fields in UpdateMetadata and
> potentially
> > > > > > Metadata requests too?
> > >
> > > We had some discussion earlier about how metadata responses to clients
> are
> > > getting too large, in part because they include a lot of information
> that
> > > most clients don't need (such as the ISR).  I think reassignment
> > > information definitely falls in the category of something a client
> doesn't
> > > need to know, so we shouldn't include it.
> > >
> > > A program like CruiseControl, or the command-line reassignment program,
> > > just wants to get the most up-to-date information about the state of
> > > reassigning partitions.  The MetadataRequest API wouldn't deliver that,
> > > because there are inherently delays in how we propagate metadata to
> > > brokers.  That's why the ListPartitionReassignments API is a better
> choice
> > > for those programs.  So I think if we added this information to the
> > > MetadataResponse, nobody would actually use it, and it would just use
> up
> > > more bandwidth.
> > >
> > > Of course, we can always revisit this later if we find a scenario
> where a
> > > producer or consumer would actually care about this.  But I think we
> should
> > > default to not adding stuff to the metadata response if we don't have a
> > > good use case in mind.
> > >
> > > > > > 11. "If a new reassignment is issued during an on-going one, we
> > > cancel the
> > > > > > current one by emptying out both AR and RR, constructing them
> from
> > > (the
> > > > > > updated from the last-reassignment) R and TR, and starting
> anew." In
> > > this
> > > > > > case, it seems that the controller needs to issue a StopReplica
> > > request to
> > > > > > remove those unneeded replicas.
> > >
> > > Good catch.  Yes, we should document this in the  KIP.
> > >
> > > > > > 12. "Essentially, once a cancellation is called we subtract AR
> from
> > > R,
> > > > > > empty out both AR and RR, and send LeaderAndIsr requests to
> cancel
> > > the
> > > > > > replica movements that have not yet completed." Similar to the
> > > above, it
> > > > > > seems the controller needs to issue a StopReplica request to
> remove
> > > those
> > > > > > unneeded replicas.
> > >
> > > Right.  Let's add this.
> > >
> > > > > > 13. Since we changed the format of the topics/[topic] zNode,
> should
> > > we bump
> > > > > > up the version number in the json value?
> > >
> > > The change to the zNode is backwards compatible, though.  Older brokers
> > > will continue to work, but just ignore the new fields.  If we bump that
> > > version number, then downgrades will require hand-editing zookeeper.
> (Of
> > > course downgrade isn't officially supported, but it would be nice not
> to
> > > break it if we don't need to...)  Changing the version number would
> also
> > > create problems during a rolling upgrade.
> > >
> > > best,
> > > Colin
> > >
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cmccabe@apache.org
> >
> > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass,
> Robert
> > > > > Barrett,
> > > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, Jason
> > > > > Gustafson,
> > > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > > >
> > > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > > sql_consulting@yahoo.com
> > > > > > > .invalid>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > >  +1 (non-binding)
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks for addressing the comments.
> > > > > > > > > George
> > > > > > > > >
> > > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen
> Shapira <
> > > > > > > > > gwen@confluent.io> wrote:
> > > > > > > > >
> > > > > > > > >  Renewing my +1, thank you Colin and Stan for working
> through
> > > all
> > > > > the
> > > > > > > > > questions, edge cases, requests and alternatives. We ended
> up
> > > with
> > > > > a
> > > > > > > > > great protocol.
> > > > > > > > >
> > > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > > >
> > > > > > > > > > -Jason
> > > > > > > > > >
> > > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > > cmccabe@apache.org
> > > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to reflect
> the
> > > fact
> > > > > > that
> > > > > > > > > we've
> > > > > > > > > > > made significant changes.  The new vote will go for 3
> days
> > > as
> > > > > > > usual.
> > > > > > > > > > >
> > > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski
> wrote:
> > > > > > > > > > > > Hey everybody,
> > > > > > > > > > > >
> > > > > > > > > > > > We have further iterated on the KIP in the
> accompanying
> > > > > > > discussion
> > > > > > > > > thread
> > > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > > >
> > > > > > > > > > > > Some notable changes:
> > > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > > - we will internally use two collections to
> represent a
> > > > > > > reassignment
> > > > > > > > > -
> > > > > > > > > > > > "addingReplicas" and "removingReplicas".
> LeaderAndIsr has
> > > > > been
> > > > > > > > > updated
> > > > > > > > > > > > accordingly
> > > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > > collection,
> > > > > > > but
> > > > > > > > > the
> > > > > > > > > > > > List API will now return three separate collections
> - the
> > > > > full
> > > > > > > > > replica
> > > > > > > > > > > set,
> > > > > > > > > > > > the replicas we are adding as part of this
> reassignment
> > > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > > > > > > > - cancellation of a reassignment now means a proper
> > > rollback
> > > > > of
> > > > > > > the
> > > > > > > > > > > > assignment to its original state prior to the API
> call
> > > > > > > > > > > >
> > > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Stanislav
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > > cmccabe@apache.org
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi George,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for taking a look.  I am working on getting
> a PR
> > > > > done
> > > > > > > as a
> > > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then we'll
> > > finish up
> > > > > > the
> > > > > > > > > vote.
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > > (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > What is the estimated timeline to have this
> > > implemented?
> > > > > > If
> > > > > > > any
> > > > > > > > > help
> > > > > > > > > > > > > > is needed in the implementation of cancelling
> > > > > > > reassignments,  I
> > > > > > > > > can
> > > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > George
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT,
> Colin
> > > > > McCabe
> > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of
> individual
> > > > > > > partitions to
> > > > > > > > > be
> > > > > > > > > > > > > > cancelled.  I think it's very important for these
> > > > > > operations
> > > > > > > to
> > > > > > > > > be at
> > > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the updated KIP.  It has very good
> > > > > > improvements
> > > > > > > of
> > > > > > > > > Kafka
> > > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > One question, looks like the KIP includes the
> > > > > > Cancellation
> > > > > > > of
> > > > > > > > > > > > > > > individual pending reassignments as well when
> the
> > > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has empty
> > > replicas
> > > > > for
> > > > > > > the
> > > > > > > > > > > > > > > topic/partition. Will you also be implementing
> the
> > > the
> > > > > > > > > partition
> > > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,
> it
> > > will
> > > > > > make
> > > > > > > > > KIP-236
> > > > > > > > > > > (it
> > > > > > > > > > > > > > > has PR already) trivial, since the cancel all
> > > pending
> > > > > > > > > > > reassignments,
> > > > > > > > > > > > > > > one just needs to do a
> > > ListPartitionRessignmentRequest,
> > > > > > > then
> > > > > > > > > submit
> > > > > > > > > > > > > > > empty replicas for all those topic/partitions
> in
> > > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > George
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT,
> Colin
> > > > > McCabe
> > > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe
> > > wrote:
> > > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason
> Gustafson
> > > > > wrote:
> > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I think storing reassignment state at the
> > > partition
> > > > > > > level
> > > > > > > > > is
> > > > > > > > > > > the
> > > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > > and I also agree that replicas should
> > > understand
> > > > > that
> > > > > > > > > there is
> > > > > > > > > > > a
> > > > > > > > > > > > > > > > > reassignment in progress. This makes
> KIP-352 a
> > > > > > trivial
> > > > > > > > > > > follow-up
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > example. The only doubt I have is whether
> the
> > > > > leader
> > > > > > > and
> > > > > > > > > isr
> > > > > > > > > > > znode
> > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > right place to store the target
> reassignment.
> > > It
> > > > > is a
> > > > > > > bit
> > > > > > > > > odd
> > > > > > > > > > > to
> > > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > > target assignment in a separate place from
> the
> > > > > > current
> > > > > > > > > > > assignment,
> > > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > > assume the thinking is probably that
> although
> > > the
> > > > > > > current
> > > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > > probably be in the leader and isr znode as
> > > well, it
> > > > > > is
> > > > > > > > > hard to
> > > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > > state in a compatible way. Is that right?
> But
> > > if we
> > > > > > > have no
> > > > > > > > > > > plan
> > > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > > the assignment znode, do you see a
> downside to
> > > > > > storing
> > > > > > > the
> > > > > > > > > > > target
> > > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > That's a good point -- it's probably better
> to
> > > keep
> > > > > the
> > > > > > > > > target
> > > > > > > > > > > > > > > > assignment in the same znode as the current
> > > > > assignment,
> > > > > > > for
> > > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I took another look at this, and I think we
> should
> > > > > stick
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > > > > initial proposal of putting the reassignment
> state
> > > into
> > > > > > > > > > > > > > >
> > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > > The
> > > > > > > > > > > reason is
> > > > > > > > > > > > > > > because we'll want to bump the leader epoch
> for the
> > > > > > > partition
> > > > > > > > > when
> > > > > > > > > > > > > > > changing the reassignment state, and the leader
> > > epoch
> > > > > > > resides
> > > > > > > > > in
> > > > > > > > > > > that
> > > > > > > > > > > > > > > znode anyway.  I agree there is some
> inconsistency
> > > > > here,
> > > > > > > but
> > > > > > > > > so be
> > > > > > > > > > > it:
> > > > > > > > > > > > > > > if we were to greenfield these zookeeper data
> > > > > structures,
> > > > > > > we
> > > > > > > > > might
> > > > > > > > > > > do
> > > > > > > > > > > > > > > it differently, but the proposed scheme will
> work
> > > fine
> > > > > > and
> > > > > > > be
> > > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > > It's the current assignment we're altering,
> > > right?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments reads
> a
> > > little
> > > > > > > > > better, and
> > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1.  I've changed the RPC and API name in the
> wiki.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2. Does this change affect the Metadata
> API? In
> > > > > other
> > > > > > > > > words,
> > > > > > > > > > > are
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > aware of reassignments? If so, then we
> probably
> > > > > need
> > > > > > a
> > > > > > > > > change
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > UpdateMetadata as well. The only
> alternative I
> > > can
> > > > > > > think of
> > > > > > > > > > > would
> > > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > > represent the replica set in the Metadata
> > > request
> > > > > as
> > > > > > > the
> > > > > > > > > union
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > current and target replicas, but I can't
> think
> > > of
> > > > > any
> > > > > > > > > benefit
> > > > > > > > > > > to
> > > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > > reassignments. Note that if we did this, we
> > > > > probably
> > > > > > > > > wouldn't
> > > > > > > > > > > need
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I thought about this a bit... and I think on
> > > balance,
> > > > > > > you're
> > > > > > > > > > > right.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > should keep this information together with
> the
> > > > > replica
> > > > > > > > > nodes, isr
> > > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > > information is
> > > > > > > > > available in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > > >  However, I do think in order to do this,
> we'll
> > > need
> > > > > a
> > > > > > > flag
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > MetadataRequest that specifiies "only show me
> > > > > > reassigning
> > > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I revisited this, and I think we should stick
> with
> > > the
> > > > > > > original
> > > > > > > > > > > > > > > proposal of having a separate
> > > > > ListPartitionReassignments
> > > > > > > API.
> > > > > > > > > > > There
> > > > > > > > > > > > > > > really is no use case where the Producer or
> > > Consumer
> > > > > > needs
> > > > > > > to
> > > > > > > > > know
> > > > > > > > > > > > > > > about a reassignment.  They should just be
> notified
> > > > > when
> > > > > > > the
> > > > > > > > > set of
> > > > > > > > > > > > > > > partitions changes, which doesn't require
> changes
> > > to
> > > > > > > > > > > > > > > MetadataRequest/Response.  The Admin client
> only
> > > cares
> > > > > if
> > > > > > > > > someone
> > > > > > > > > > > is
> > > > > > > > > > > > > > > managing the reassignment.  So adding this
> state
> > > to the
> > > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > > benefit.  In
> > > > > > the
> > > > > > > > > common
> > > > > > > > > > > > > case
> > > > > > > > > > > > > > > where there is no ongoing reassignment, it
> would
> > > be 4
> > > > > > > bytes per
> > > > > > > > > > > > > > > partition of extra overhead in the
> > > MetadataResponse.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > In general, I think we have a problem of
> > > oversharing in
> > > > > > the
> > > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 100x
> the
> > > number
> > > > > > of
> > > > > > > > > > > partitions
> > > > > > > > > > > > > > > we support, we'll need to get stricter about
> giving
> > > > > > clients
> > > > > > > > > only
> > > > > > > > > > > the
> > > > > > > > > > > > > > > information they actually need, about the
> > > partitions
> > > > > they
> > > > > > > > > actually
> > > > > > > > > > > > > care
> > > > > > > > > > > > > > > about.  Reassignment state clearly falls in the
> > > > > category
> > > > > > of
> > > > > > > > > state
> > > > > > > > > > > that
> > > > > > > > > > > > > > > isn't needed by clients (except very
> specialized
> > > > > > > rebalancing
> > > > > > > > > > > programs).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Another important consideration here is that
> > > someone
> > > > > > > managing
> > > > > > > > > an
> > > > > > > > > > > > > > > ongoing reassignment wants the most up-to-date
> > > > > > information,
> > > > > > > > > which
> > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > be found on the controller.  Therefore adding
> this
> > > > > state
> > > > > > to
> > > > > > > > > > > listTopics
> > > > > > > > > > > > > > > or describeTopics, which could contact any
> node in
> > > the
> > > > > > > > > cluster, is
> > > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Finally, adding this to listTopics or
> > > describeTopics
> > > > > > feels
> > > > > > > > > like a
> > > > > > > > > > > > > warty
> > > > > > > > > > > > > > > API.  It's an extra boolean which interacts
> with
> > > other
> > > > > > > extra
> > > > > > > > > > > booleans
> > > > > > > > > > > > > > > like "show internal", etc. in weird ways.  I
> think
> > > a
> > > > > > > separate
> > > > > > > > > API
> > > > > > > > > > > is
> > > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 3. As replicas come into sync, they will
> join
> > > the
> > > > > > ISR.
> > > > > > > > > Will we
> > > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > > target replicas joining the ISR before
> taking
> > > the
> > > > > > > replica
> > > > > > > > > out
> > > > > > > > > > > of
> > > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > > replicas set? Also, I assume that target
> > > replicas
> > > > > can
> > > > > > > > > still be
> > > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > We'll take a replica out of the target
> replicas
> > > set
> > > > > as
> > > > > > > soon
> > > > > > > > > as
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify this
> in
> > > the
> > > > > KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 4. Probably useful to mention permissions
> for
> > > the
> > > > > new
> > > > > > > APIs.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Good point.  I think
> alterPartitionAssignments
> > > should
> > > > > > > require
> > > > > > > > > > > ALTER
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions will be
> > > > > > unchanged.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Jason
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen
> Shapira <
> > > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > Looks great, and will be awesome to have
> > > this new
> > > > > > > > > capability.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin
> McCabe
> > > <
> > > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > > Create
> > > > > an
> > > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > > Replica Reassignment.  I think this
> KIP is
> > > > > > > important
> > > > > > > > > since
> > > > > > > > > > > it
> > > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > > many follow-on improvements to Kafka
> > > > > reassignment
> > > > > > > (see
> > > > > > > > > the
> > > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > > discussions
> > > > > > we've
> > > > > > > had
> > > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > > reassignment).  It also furthers the
> > > important
> > > > > > > KIP-4
> > > > > > > > > goal
> > > > > > > > > > > of
> > > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I made a few changes based on the
> > > discussion in
> > > > > > the
> > > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > > Robert suggested, I removed the need to
> > > > > > explicitly
> > > > > > > > > cancel a
> > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > for a partition before setting up a
> > > different
> > > > > > > > > reassignment
> > > > > > > > > > > for
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > specific partition.  I also simplified
> the
> > > API
> > > > > a
> > > > > > > bit by
> > > > > > > > > > > adding
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > PartitionReassignment class which is
> used
> > > by
> > > > > both
> > > > > > > the
> > > > > > > > > alter
> > > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I modified the proposal so that we now
> > > > > deprecate
> > > > > > > the
> > > > > > > > > old
> > > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > > rather than removing it completely.
> That
> > > > > should
> > > > > > > give
> > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > > tools some time to transition to the
> new
> > > API.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > To clarify a question Viktor asked, I
> > > added a
> > > > > > note
> > > > > > > > > that the
> > > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will now
> use a
> > > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > > > > > > > Follow us: Twitter <
> > > > > > https://twitter.com/ConfluentInc>
> > > > > > > |
> > > > > > > > > blog
> > > > > > > > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > Gwen Shapira
> > > > > > > > > Product Manager | Confluent
> > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > Follow us: Twitter | blog
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
On Thu, Aug 1, 2019, at 12:00, Jun Rao wrote:
> Hi, Colin,
> 
> 10. Sounds good.
> 
> 13. Our current convention is to bump up the version of ZK value if there
> is any format change. For example, we have bumped up the version of the
> value in /brokers/ids/nnn multiple times and all of those changes are
> compatible (just adding new fields). This has the slight benefit that it
> makes it clear there is a format change. Rolling upgrades and downgrades
> can still be supported with the version bump. For example, if you downgrade
> from a compatible change, you can leave the new format in ZK and the old
> code will only pick up fields relevant to the old version. Upgrade will be
> controlled by inter broker protocol.

Hmm.  If we bump that ZK node version, we will need a new inter-broker protocol version.  We also need to return UnsupportedVersionException from the alterPartitionReassignments and listPartitionReassignments APIs when the IBP is too low.  This sounds doable, although we might need a release note that upgrading the IBP is necessary to allow reassignment operations after an upgrade.

best,
Colin

> 
> Thanks,
> 
> Jun
> 
> On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi Jun,
> >
> > Thanks for taking another look at this.
> >
> > On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > > Hi, Stan,
> > >
> > > Thanks for the explanation.
> > >
> > > 10. If those new fields in LeaderAndIsr are only needed for future work,
> > > perhaps they should be added when we do the future work instead of now?
> >
> > I think this ties in with one of the big goals of this KIP, making it
> > possible to distinguish reassigning replicas from normal replicas.  This is
> > the key to follow-on work like being able to ensure that partitions with a
> > reassignment don't get falsely flagged as under-replicated in the metrics,
> > or implementing reassignment quotas that don't accidentally affect normal
> > replication traffic when a replica falls out of the ISR.
> >
> > For these follow-on improvements, we need to have that information in
> > LeaderAndIsrRequest.  We could add the information in a follow-on KIP, of
> > course, but then all the improvements are blocked on that follow-on KIP.
> > That would slow things down for all of the downstream KIPs that are blocked
> > on this.
> >
> > Also, to keep things consistent, I think it would be best if the format of
> > the data in the LeaderAndIsrRequest matched the format of the data in
> > ZooKeeper.  Since we're deciding on the ZK format in this KIP, I think it
> > makes sense to also decide on the format in the LeaderAndIsrRequest.
> >
> > > > > Should we include those two fields in UpdateMetadata and potentially
> > > > > Metadata requests too?
> >
> > We had some discussion earlier about how metadata responses to clients are
> > getting too large, in part because they include a lot of information that
> > most clients don't need (such as the ISR).  I think reassignment
> > information definitely falls in the category of something a client doesn't
> > need to know, so we shouldn't include it.
> >
> > A program like CruiseControl, or the command-line reassignment program,
> > just wants to get the most up-to-date information about the state of
> > reassigning partitions.  The MetadataRequest API wouldn't deliver that,
> > because there are inherently delays in how we propagate metadata to
> > brokers.  That's why the ListPartitionReassignments API is a better choice
> > for those programs.  So I think if we added this information to the
> > MetadataResponse, nobody would actually use it, and it would just use up
> > more bandwidth.
> >
> > Of course, we can always revisit this later if we find a scenario where a
> > producer or consumer would actually care about this.  But I think we should
> > default to not adding stuff to the metadata response if we don't have a
> > good use case in mind.
> >
> > > > > 11. "If a new reassignment is issued during an on-going one, we
> > cancel the
> > > > > current one by emptying out both AR and RR, constructing them from
> > (the
> > > > > updated from the last-reassignment) R and TR, and starting anew." In
> > this
> > > > > case, it seems that the controller needs to issue a StopReplica
> > request to
> > > > > remove those unneeded replicas.
> >
> > Good catch.  Yes, we should document this in the  KIP.
> >
> > > > > 12. "Essentially, once a cancellation is called we subtract AR from
> > R,
> > > > > empty out both AR and RR, and send LeaderAndIsr requests to cancel
> > the
> > > > > replica movements that have not yet completed." Similar to the
> > above, it
> > > > > seems the controller needs to issue a StopReplica request to remove
> > those
> > > > > unneeded replicas.
> >
> > Right.  Let's add this.
> >
> > > > > 13. Since we changed the format of the topics/[topic] zNode, should
> > we bump
> > > > > up the version number in the json value?
> >
> > The change to the zNode is backwards compatible, though.  Older brokers
> > will continue to work, but just ignore the new fields.  If we bump that
> > version number, then downgrades will require hand-editing zookeeper.  (Of
> > course downgrade isn't officially supported, but it would be nice not to
> > break it if we don't need to...)  Changing the version number would also
> > create problems during a rolling upgrade.
> >
> > best,
> > Colin
> >
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > With three non-binding +1 votes from Viktor Somogyi-Vass, Robert
> > > > Barrett,
> > > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, Jason
> > > > Gustafson,
> > > > > > and myself, the vote passes.  Thanks, everyone!
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > > >
> > > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> > sql_consulting@yahoo.com
> > > > > > .invalid>
> > > > > > > wrote:
> > > > > > >
> > > > > > > >  +1 (non-binding)
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks for addressing the comments.
> > > > > > > > George
> > > > > > > >
> > > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > > > > > gwen@confluent.io> wrote:
> > > > > > > >
> > > > > > > >  Renewing my +1, thank you Colin and Stan for working through
> > all
> > > > the
> > > > > > > > questions, edge cases, requests and alternatives. We ended up
> > with
> > > > a
> > > > > > > > great protocol.
> > > > > > > >
> > > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > > jason@confluent.io>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > > >
> > > > > > > > > -Jason
> > > > > > > > >
> > > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> > cmccabe@apache.org
> > > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks, Stanislav.  Let's restart the vote to reflect the
> > fact
> > > > > that
> > > > > > > > we've
> > > > > > > > > > made significant changes.  The new vote will go for 3 days
> > as
> > > > > > usual.
> > > > > > > > > >
> > > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > > > > > > > Hey everybody,
> > > > > > > > > > >
> > > > > > > > > > > We have further iterated on the KIP in the accompanying
> > > > > > discussion
> > > > > > > > thread
> > > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > > >
> > > > > > > > > > > Some notable changes:
> > > > > > > > > > > - we will store reassignment information in the
> > > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > > - we will internally use two collections to represent a
> > > > > > reassignment
> > > > > > > > -
> > > > > > > > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has
> > > > been
> > > > > > > > updated
> > > > > > > > > > > accordingly
> > > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > > collection,
> > > > > > but
> > > > > > > > the
> > > > > > > > > > > List API will now return three separate collections - the
> > > > full
> > > > > > > > replica
> > > > > > > > > > set,
> > > > > > > > > > > the replicas we are adding as part of this reassignment
> > > > > > > > > > ("addingReplicas")
> > > > > > > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > > > > > > - cancellation of a reassignment now means a proper
> > rollback
> > > > of
> > > > > > the
> > > > > > > > > > > assignment to its original state prior to the API call
> > > > > > > > > > >
> > > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Stanislav
> > > > > > > > > > >
> > > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > > cmccabe@apache.org
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi George,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for taking a look.  I am working on getting a PR
> > > > done
> > > > > > as a
> > > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then we'll
> > finish up
> > > > > the
> > > > > > > > vote.
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > >
> > > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > > (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > What is the estimated timeline to have this
> > implemented?
> > > > > If
> > > > > > any
> > > > > > > > help
> > > > > > > > > > > > > is needed in the implementation of cancelling
> > > > > > reassignments,  I
> > > > > > > > can
> > > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > George
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin
> > > > McCabe
> > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >  Hi George,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Yes, KIP-455 allows the reassignment of individual
> > > > > > partitions to
> > > > > > > > be
> > > > > > > > > > > > > cancelled.  I think it's very important for these
> > > > > operations
> > > > > > to
> > > > > > > > be at
> > > > > > > > > > > > > the partition level.
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the updated KIP.  It has very good
> > > > > improvements
> > > > > > of
> > > > > > > > Kafka
> > > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > One question, looks like the KIP includes the
> > > > > Cancellation
> > > > > > of
> > > > > > > > > > > > > > individual pending reassignments as well when the
> > > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has empty
> > replicas
> > > > for
> > > > > > the
> > > > > > > > > > > > > > topic/partition. Will you also be implementing the
> > the
> > > > > > > > partition
> > > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,  it
> > will
> > > > > make
> > > > > > > > KIP-236
> > > > > > > > > > (it
> > > > > > > > > > > > > > has PR already) trivial, since the cancel all
> > pending
> > > > > > > > > > reassignments,
> > > > > > > > > > > > > > one just needs to do a
> > ListPartitionRessignmentRequest,
> > > > > > then
> > > > > > > > submit
> > > > > > > > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > George
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin
> > > > McCabe
> > > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe
> > wrote:
> > > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson
> > > > wrote:
> > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I think storing reassignment state at the
> > partition
> > > > > > level
> > > > > > > > is
> > > > > > > > > > the
> > > > > > > > > > > > right move
> > > > > > > > > > > > > > > > and I also agree that replicas should
> > understand
> > > > that
> > > > > > > > there is
> > > > > > > > > > a
> > > > > > > > > > > > > > > > reassignment in progress. This makes KIP-352 a
> > > > > trivial
> > > > > > > > > > follow-up
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > example. The only doubt I have is whether the
> > > > leader
> > > > > > and
> > > > > > > > isr
> > > > > > > > > > znode
> > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > right place to store the target reassignment.
> > It
> > > > is a
> > > > > > bit
> > > > > > > > odd
> > > > > > > > > > to
> > > > > > > > > > > > keep the
> > > > > > > > > > > > > > > > target assignment in a separate place from the
> > > > > current
> > > > > > > > > > assignment,
> > > > > > > > > > > > right? I
> > > > > > > > > > > > > > > > assume the thinking is probably that although
> > the
> > > > > > current
> > > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > > probably be in the leader and isr znode as
> > well, it
> > > > > is
> > > > > > > > hard to
> > > > > > > > > > > > move the
> > > > > > > > > > > > > > > > state in a compatible way. Is that right? But
> > if we
> > > > > > have no
> > > > > > > > > > plan
> > > > > > > > > > > > to remove
> > > > > > > > > > > > > > > > the assignment znode, do you see a downside to
> > > > > storing
> > > > > > the
> > > > > > > > > > target
> > > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > That's a good point -- it's probably better to
> > keep
> > > > the
> > > > > > > > target
> > > > > > > > > > > > > > > assignment in the same znode as the current
> > > > assignment,
> > > > > > for
> > > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I took another look at this, and I think we should
> > > > stick
> > > > > > with
> > > > > > > > the
> > > > > > > > > > > > > > initial proposal of putting the reassignment state
> > into
> > > > > > > > > > > > > >
> > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > > The
> > > > > > > > > > reason is
> > > > > > > > > > > > > > because we'll want to bump the leader epoch for the
> > > > > > partition
> > > > > > > > when
> > > > > > > > > > > > > > changing the reassignment state, and the leader
> > epoch
> > > > > > resides
> > > > > > > > in
> > > > > > > > > > that
> > > > > > > > > > > > > > znode anyway.  I agree there is some inconsistency
> > > > here,
> > > > > > but
> > > > > > > > so be
> > > > > > > > > > it:
> > > > > > > > > > > > > > if we were to greenfield these zookeeper data
> > > > structures,
> > > > > > we
> > > > > > > > might
> > > > > > > > > > do
> > > > > > > > > > > > > > it differently, but the proposed scheme will work
> > fine
> > > > > and
> > > > > > be
> > > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > > It's the current assignment we're altering,
> > right?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments reads a
> > little
> > > > > > > > better, and
> > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2. Does this change affect the Metadata API? In
> > > > other
> > > > > > > > words,
> > > > > > > > > > are
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > aware of reassignments? If so, then we probably
> > > > need
> > > > > a
> > > > > > > > change
> > > > > > > > > > to
> > > > > > > > > > > > > > > > UpdateMetadata as well. The only alternative I
> > can
> > > > > > think of
> > > > > > > > > > would
> > > > > > > > > > > > be to
> > > > > > > > > > > > > > > > represent the replica set in the Metadata
> > request
> > > > as
> > > > > > the
> > > > > > > > union
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > current and target replicas, but I can't think
> > of
> > > > any
> > > > > > > > benefit
> > > > > > > > > > to
> > > > > > > > > > > > hiding
> > > > > > > > > > > > > > > > reassignments. Note that if we did this, we
> > > > probably
> > > > > > > > wouldn't
> > > > > > > > > > need
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I thought about this a bit... and I think on
> > balance,
> > > > > > you're
> > > > > > > > > > right.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > > should keep this information together with the
> > > > replica
> > > > > > > > nodes, isr
> > > > > > > > > > > > > > > nodes, and offline replicas, and that
> > information is
> > > > > > > > available in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > > >  However, I do think in order to do this, we'll
> > need
> > > > a
> > > > > > flag
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > > MetadataRequest that specifiies "only show me
> > > > > reassigning
> > > > > > > > > > > > partitions".
> > > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I revisited this, and I think we should stick with
> > the
> > > > > > original
> > > > > > > > > > > > > > proposal of having a separate
> > > > ListPartitionReassignments
> > > > > > API.
> > > > > > > > > > There
> > > > > > > > > > > > > > really is no use case where the Producer or
> > Consumer
> > > > > needs
> > > > > > to
> > > > > > > > know
> > > > > > > > > > > > > > about a reassignment.  They should just be notified
> > > > when
> > > > > > the
> > > > > > > > set of
> > > > > > > > > > > > > > partitions changes, which doesn't require changes
> > to
> > > > > > > > > > > > > > MetadataRequest/Response.  The Admin client only
> > cares
> > > > if
> > > > > > > > someone
> > > > > > > > > > is
> > > > > > > > > > > > > > managing the reassignment.  So adding this state
> > to the
> > > > > > > > > > > > > > MetadataResponse adds overhead for no real
> > benefit.  In
> > > > > the
> > > > > > > > common
> > > > > > > > > > > > case
> > > > > > > > > > > > > > where there is no ongoing reassignment, it would
> > be 4
> > > > > > bytes per
> > > > > > > > > > > > > > partition of extra overhead in the
> > MetadataResponse.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In general, I think we have a problem of
> > oversharing in
> > > > > the
> > > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 100x the
> > number
> > > > > of
> > > > > > > > > > partitions
> > > > > > > > > > > > > > we support, we'll need to get stricter about giving
> > > > > clients
> > > > > > > > only
> > > > > > > > > > the
> > > > > > > > > > > > > > information they actually need, about the
> > partitions
> > > > they
> > > > > > > > actually
> > > > > > > > > > > > care
> > > > > > > > > > > > > > about.  Reassignment state clearly falls in the
> > > > category
> > > > > of
> > > > > > > > state
> > > > > > > > > > that
> > > > > > > > > > > > > > isn't needed by clients (except very specialized
> > > > > > rebalancing
> > > > > > > > > > programs).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Another important consideration here is that
> > someone
> > > > > > managing
> > > > > > > > an
> > > > > > > > > > > > > > ongoing reassignment wants the most up-to-date
> > > > > information,
> > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > be found on the controller.  Therefore adding this
> > > > state
> > > > > to
> > > > > > > > > > listTopics
> > > > > > > > > > > > > > or describeTopics, which could contact any node in
> > the
> > > > > > > > cluster, is
> > > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Finally, adding this to listTopics or
> > describeTopics
> > > > > feels
> > > > > > > > like a
> > > > > > > > > > > > warty
> > > > > > > > > > > > > > API.  It's an extra boolean which interacts with
> > other
> > > > > > extra
> > > > > > > > > > booleans
> > > > > > > > > > > > > > like "show internal", etc. in weird ways.  I think
> > a
> > > > > > separate
> > > > > > > > API
> > > > > > > > > > is
> > > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 3. As replicas come into sync, they will join
> > the
> > > > > ISR.
> > > > > > > > Will we
> > > > > > > > > > > > await all
> > > > > > > > > > > > > > > > target replicas joining the ISR before taking
> > the
> > > > > > replica
> > > > > > > > out
> > > > > > > > > > of
> > > > > > > > > > > > the target
> > > > > > > > > > > > > > > > replicas set? Also, I assume that target
> > replicas
> > > > can
> > > > > > > > still be
> > > > > > > > > > > > elected as
> > > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We'll take a replica out of the target replicas
> > set
> > > > as
> > > > > > soon
> > > > > > > > as
> > > > > > > > > > that
> > > > > > > > > > > > > > > replica is in the ISR.  Let me clarify this in
> > the
> > > > KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 4. Probably useful to mention permissions for
> > the
> > > > new
> > > > > > APIs.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Good point.  I think alterPartitionAssignments
> > should
> > > > > > require
> > > > > > > > > > ALTER
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions will be
> > > > > unchanged.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Jason
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > Looks great, and will be awesome to have
> > this new
> > > > > > > > capability.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe
> > <
> > > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> > Create
> > > > an
> > > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > > Replica Reassignment.  I think this KIP is
> > > > > > important
> > > > > > > > since
> > > > > > > > > > it
> > > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > > many follow-on improvements to Kafka
> > > > reassignment
> > > > > > (see
> > > > > > > > the
> > > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > > section, plus a lot of the other
> > discussions
> > > > > we've
> > > > > > had
> > > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > > reassignment).  It also furthers the
> > important
> > > > > > KIP-4
> > > > > > > > goal
> > > > > > > > > > of
> > > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I made a few changes based on the
> > discussion in
> > > > > the
> > > > > > > > > > [DISCUSS]
> > > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > > Robert suggested, I removed the need to
> > > > > explicitly
> > > > > > > > cancel a
> > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > > for a partition before setting up a
> > different
> > > > > > > > reassignment
> > > > > > > > > > for
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > specific partition.  I also simplified the
> > API
> > > > a
> > > > > > bit by
> > > > > > > > > > adding
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > PartitionReassignment class which is used
> > by
> > > > both
> > > > > > the
> > > > > > > > alter
> > > > > > > > > > > > and list
> > > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I modified the proposal so that we now
> > > > deprecate
> > > > > > the
> > > > > > > > old
> > > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > > rather than removing it completely.  That
> > > > should
> > > > > > give
> > > > > > > > > > external
> > > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > > tools some time to transition to the new
> > API.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > To clarify a question Viktor asked, I
> > added a
> > > > > note
> > > > > > > > that the
> > > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > > > > > > > --bootstrap-server
> > > > > > > > > > > > argument
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > > > > > > Follow us: Twitter <
> > > > > https://twitter.com/ConfluentInc>
> > > > > > |
> > > > > > > > blog
> > > > > > > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > Gwen Shapira
> > > > > > > > Product Manager | Confluent
> > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > Follow us: Twitter | blog
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Jun Rao <ju...@confluent.io>.
Hi, Colin,

10. Sounds good.

13. Our current convention is to bump up the version of ZK value if there
is any format change. For example, we have bumped up the version of the
value in /brokers/ids/nnn multiple times and all of those changes are
compatible (just adding new fields). This has the slight benefit that it
makes it clear there is a format change. Rolling upgrades and downgrades
can still be supported with the version bump. For example, if you downgrade
from a compatible change, you can leave the new format in ZK and the old
code will only pick up fields relevant to the old version. Upgrade will be
controlled by inter broker protocol.

Thanks,

Jun

On Wed, Jul 31, 2019 at 1:22 PM Colin McCabe <cm...@apache.org> wrote:

> Hi Jun,
>
> Thanks for taking another look at this.
>
> On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> > Hi, Stan,
> >
> > Thanks for the explanation.
> >
> > 10. If those new fields in LeaderAndIsr are only needed for future work,
> > perhaps they should be added when we do the future work instead of now?
>
> I think this ties in with one of the big goals of this KIP, making it
> possible to distinguish reassigning replicas from normal replicas.  This is
> the key to follow-on work like being able to ensure that partitions with a
> reassignment don't get falsely flagged as under-replicated in the metrics,
> or implementing reassignment quotas that don't accidentally affect normal
> replication traffic when a replica falls out of the ISR.
>
> For these follow-on improvements, we need to have that information in
> LeaderAndIsrRequest.  We could add the information in a follow-on KIP, of
> course, but then all the improvements are blocked on that follow-on KIP.
> That would slow things down for all of the downstream KIPs that are blocked
> on this.
>
> Also, to keep things consistent, I think it would be best if the format of
> the data in the LeaderAndIsrRequest matched the format of the data in
> ZooKeeper.  Since we're deciding on the ZK format in this KIP, I think it
> makes sense to also decide on the format in the LeaderAndIsrRequest.
>
> > > > Should we include those two fields in UpdateMetadata and potentially
> > > > Metadata requests too?
>
> We had some discussion earlier about how metadata responses to clients are
> getting too large, in part because they include a lot of information that
> most clients don't need (such as the ISR).  I think reassignment
> information definitely falls in the category of something a client doesn't
> need to know, so we shouldn't include it.
>
> A program like CruiseControl, or the command-line reassignment program,
> just wants to get the most up-to-date information about the state of
> reassigning partitions.  The MetadataRequest API wouldn't deliver that,
> because there are inherently delays in how we propagate metadata to
> brokers.  That's why the ListPartitionReassignments API is a better choice
> for those programs.  So I think if we added this information to the
> MetadataResponse, nobody would actually use it, and it would just use up
> more bandwidth.
>
> Of course, we can always revisit this later if we find a scenario where a
> producer or consumer would actually care about this.  But I think we should
> default to not adding stuff to the metadata response if we don't have a
> good use case in mind.
>
> > > > 11. "If a new reassignment is issued during an on-going one, we
> cancel the
> > > > current one by emptying out both AR and RR, constructing them from
> (the
> > > > updated from the last-reassignment) R and TR, and starting anew." In
> this
> > > > case, it seems that the controller needs to issue a StopReplica
> request to
> > > > remove those unneeded replicas.
>
> Good catch.  Yes, we should document this in the  KIP.
>
> > > > 12. "Essentially, once a cancellation is called we subtract AR from
> R,
> > > > empty out both AR and RR, and send LeaderAndIsr requests to cancel
> the
> > > > replica movements that have not yet completed." Similar to the
> above, it
> > > > seems the controller needs to issue a StopReplica request to remove
> those
> > > > unneeded replicas.
>
> Right.  Let's add this.
>
> > > > 13. Since we changed the format of the topics/[topic] zNode, should
> we bump
> > > > up the version number in the json value?
>
> The change to the zNode is backwards compatible, though.  Older brokers
> will continue to work, but just ignore the new fields.  If we bump that
> version number, then downgrades will require hand-editing zookeeper.  (Of
> course downgrade isn't officially supported, but it would be nice not to
> break it if we don't need to...)  Changing the version number would also
> create problems during a rolling upgrade.
>
> best,
> Colin
>
> > > >
> > > > Jun
> > > >
> > > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cm...@apache.org>
> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > With three non-binding +1 votes from Viktor Somogyi-Vass, Robert
> > > Barrett,
> > > > > and George Li, and 3 binding +1 votes from Gwen Shapira, Jason
> > > Gustafson,
> > > > > and myself, the vote passes.  Thanks, everyone!
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > > +1 (non-binding). Thanks for the KIP!
> > > > > >
> > > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <
> sql_consulting@yahoo.com
> > > > > .invalid>
> > > > > > wrote:
> > > > > >
> > > > > > >  +1 (non-binding)
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Thanks for addressing the comments.
> > > > > > > George
> > > > > > >
> > > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > > > > gwen@confluent.io> wrote:
> > > > > > >
> > > > > > >  Renewing my +1, thank you Colin and Stan for working through
> all
> > > the
> > > > > > > questions, edge cases, requests and alternatives. We ended up
> with
> > > a
> > > > > > > great protocol.
> > > > > > >
> > > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > > jason@confluent.io>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > > >
> > > > > > > > -Jason
> > > > > > > >
> > > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <
> cmccabe@apache.org
> > > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks, Stanislav.  Let's restart the vote to reflect the
> fact
> > > > that
> > > > > > > we've
> > > > > > > > > made significant changes.  The new vote will go for 3 days
> as
> > > > > usual.
> > > > > > > > >
> > > > > > > > > I'll start with my +1 (binding).
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > > > > > > Hey everybody,
> > > > > > > > > >
> > > > > > > > > > We have further iterated on the KIP in the accompanying
> > > > > discussion
> > > > > > > thread
> > > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > > >
> > > > > > > > > > Some notable changes:
> > > > > > > > > > - we will store reassignment information in the
> > > > > > > `/brokers/topics/[topic]`
> > > > > > > > > > - we will internally use two collections to represent a
> > > > > reassignment
> > > > > > > -
> > > > > > > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has
> > > been
> > > > > > > updated
> > > > > > > > > > accordingly
> > > > > > > > > > - the Alter API will still use the "targetReplicas"
> > > collection,
> > > > > but
> > > > > > > the
> > > > > > > > > > List API will now return three separate collections - the
> > > full
> > > > > > > replica
> > > > > > > > > set,
> > > > > > > > > > the replicas we are adding as part of this reassignment
> > > > > > > > > ("addingReplicas")
> > > > > > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > > > > > - cancellation of a reassignment now means a proper
> rollback
> > > of
> > > > > the
> > > > > > > > > > assignment to its original state prior to the API call
> > > > > > > > > >
> > > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Stanislav
> > > > > > > > > >
> > > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > > cmccabe@apache.org
> > > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi George,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for taking a look.  I am working on getting a PR
> > > done
> > > > > as a
> > > > > > > > > > > proof-of-concept.  I'll post it soon.  Then we'll
> finish up
> > > > the
> > > > > > > vote.
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > >
> > > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > > (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > What is the estimated timeline to have this
> implemented?
> > > > If
> > > > > any
> > > > > > > help
> > > > > > > > > > > > is needed in the implementation of cancelling
> > > > > reassignments,  I
> > > > > > > can
> > > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > George
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin
> > > McCabe
> > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >  Hi George,
> > > > > > > > > > > >
> > > > > > > > > > > > Yes, KIP-455 allows the reassignment of individual
> > > > > partitions to
> > > > > > > be
> > > > > > > > > > > > cancelled.  I think it's very important for these
> > > > operations
> > > > > to
> > > > > > > be at
> > > > > > > > > > > > the partition level.
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the updated KIP.  It has very good
> > > > improvements
> > > > > of
> > > > > > > Kafka
> > > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > > >
> > > > > > > > > > > > > One question, looks like the KIP includes the
> > > > Cancellation
> > > > > of
> > > > > > > > > > > > > individual pending reassignments as well when the
> > > > > > > > > > > > > AlterPartitionReasisgnmentRequest has empty
> replicas
> > > for
> > > > > the
> > > > > > > > > > > > > topic/partition. Will you also be implementing the
> the
> > > > > > > partition
> > > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,  it
> will
> > > > make
> > > > > > > KIP-236
> > > > > > > > > (it
> > > > > > > > > > > > > has PR already) trivial, since the cancel all
> pending
> > > > > > > > > reassignments,
> > > > > > > > > > > > > one just needs to do a
> ListPartitionRessignmentRequest,
> > > > > then
> > > > > > > submit
> > > > > > > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > George
> > > > > > > > > > > > >
> > > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin
> > > McCabe
> > > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe
> wrote:
> > > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson
> > > wrote:
> > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think storing reassignment state at the
> partition
> > > > > level
> > > > > > > is
> > > > > > > > > the
> > > > > > > > > > > right move
> > > > > > > > > > > > > > > and I also agree that replicas should
> understand
> > > that
> > > > > > > there is
> > > > > > > > > a
> > > > > > > > > > > > > > > reassignment in progress. This makes KIP-352 a
> > > > trivial
> > > > > > > > > follow-up
> > > > > > > > > > > for
> > > > > > > > > > > > > > > example. The only doubt I have is whether the
> > > leader
> > > > > and
> > > > > > > isr
> > > > > > > > > znode
> > > > > > > > > > > is the
> > > > > > > > > > > > > > > right place to store the target reassignment.
> It
> > > is a
> > > > > bit
> > > > > > > odd
> > > > > > > > > to
> > > > > > > > > > > keep the
> > > > > > > > > > > > > > > target assignment in a separate place from the
> > > > current
> > > > > > > > > assignment,
> > > > > > > > > > > right? I
> > > > > > > > > > > > > > > assume the thinking is probably that although
> the
> > > > > current
> > > > > > > > > > > assignment should
> > > > > > > > > > > > > > > probably be in the leader and isr znode as
> well, it
> > > > is
> > > > > > > hard to
> > > > > > > > > > > move the
> > > > > > > > > > > > > > > state in a compatible way. Is that right? But
> if we
> > > > > have no
> > > > > > > > > plan
> > > > > > > > > > > to remove
> > > > > > > > > > > > > > > the assignment znode, do you see a downside to
> > > > storing
> > > > > the
> > > > > > > > > target
> > > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > That's a good point -- it's probably better to
> keep
> > > the
> > > > > > > target
> > > > > > > > > > > > > > assignment in the same znode as the current
> > > assignment,
> > > > > for
> > > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I took another look at this, and I think we should
> > > stick
> > > > > with
> > > > > > > the
> > > > > > > > > > > > > initial proposal of putting the reassignment state
> into
> > > > > > > > > > > > >
> /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > > The
> > > > > > > > > reason is
> > > > > > > > > > > > > because we'll want to bump the leader epoch for the
> > > > > partition
> > > > > > > when
> > > > > > > > > > > > > changing the reassignment state, and the leader
> epoch
> > > > > resides
> > > > > > > in
> > > > > > > > > that
> > > > > > > > > > > > > znode anyway.  I agree there is some inconsistency
> > > here,
> > > > > but
> > > > > > > so be
> > > > > > > > > it:
> > > > > > > > > > > > > if we were to greenfield these zookeeper data
> > > structures,
> > > > > we
> > > > > > > might
> > > > > > > > > do
> > > > > > > > > > > > > it differently, but the proposed scheme will work
> fine
> > > > and
> > > > > be
> > > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > > It's the current assignment we're altering,
> right?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > That's fair.  AlterPartitionAssigments reads a
> little
> > > > > > > better, and
> > > > > > > > > > > I'll
> > > > > > > > > > > > > > change it to that.
> > > > > > > > > > > > >
> > > > > > > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 2. Does this change affect the Metadata API? In
> > > other
> > > > > > > words,
> > > > > > > > > are
> > > > > > > > > > > clients
> > > > > > > > > > > > > > > aware of reassignments? If so, then we probably
> > > need
> > > > a
> > > > > > > change
> > > > > > > > > to
> > > > > > > > > > > > > > > UpdateMetadata as well. The only alternative I
> can
> > > > > think of
> > > > > > > > > would
> > > > > > > > > > > be to
> > > > > > > > > > > > > > > represent the replica set in the Metadata
> request
> > > as
> > > > > the
> > > > > > > union
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > > current and target replicas, but I can't think
> of
> > > any
> > > > > > > benefit
> > > > > > > > > to
> > > > > > > > > > > hiding
> > > > > > > > > > > > > > > reassignments. Note that if we did this, we
> > > probably
> > > > > > > wouldn't
> > > > > > > > > need
> > > > > > > > > > > a
> > > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I thought about this a bit... and I think on
> balance,
> > > > > you're
> > > > > > > > > right.
> > > > > > > > > > > We
> > > > > > > > > > > > > > should keep this information together with the
> > > replica
> > > > > > > nodes, isr
> > > > > > > > > > > > > > nodes, and offline replicas, and that
> information is
> > > > > > > available in
> > > > > > > > > > > the
> > > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > > >  However, I do think in order to do this, we'll
> need
> > > a
> > > > > flag
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > > MetadataRequest that specifiies "only show me
> > > > reassigning
> > > > > > > > > > > partitions".
> > > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I revisited this, and I think we should stick with
> the
> > > > > original
> > > > > > > > > > > > > proposal of having a separate
> > > ListPartitionReassignments
> > > > > API.
> > > > > > > > > There
> > > > > > > > > > > > > really is no use case where the Producer or
> Consumer
> > > > needs
> > > > > to
> > > > > > > know
> > > > > > > > > > > > > about a reassignment.  They should just be notified
> > > when
> > > > > the
> > > > > > > set of
> > > > > > > > > > > > > partitions changes, which doesn't require changes
> to
> > > > > > > > > > > > > MetadataRequest/Response.  The Admin client only
> cares
> > > if
> > > > > > > someone
> > > > > > > > > is
> > > > > > > > > > > > > managing the reassignment.  So adding this state
> to the
> > > > > > > > > > > > > MetadataResponse adds overhead for no real
> benefit.  In
> > > > the
> > > > > > > common
> > > > > > > > > > > case
> > > > > > > > > > > > > where there is no ongoing reassignment, it would
> be 4
> > > > > bytes per
> > > > > > > > > > > > > partition of extra overhead in the
> MetadataResponse.
> > > > > > > > > > > > >
> > > > > > > > > > > > > In general, I think we have a problem of
> oversharing in
> > > > the
> > > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 100x the
> number
> > > > of
> > > > > > > > > partitions
> > > > > > > > > > > > > we support, we'll need to get stricter about giving
> > > > clients
> > > > > > > only
> > > > > > > > > the
> > > > > > > > > > > > > information they actually need, about the
> partitions
> > > they
> > > > > > > actually
> > > > > > > > > > > care
> > > > > > > > > > > > > about.  Reassignment state clearly falls in the
> > > category
> > > > of
> > > > > > > state
> > > > > > > > > that
> > > > > > > > > > > > > isn't needed by clients (except very specialized
> > > > > rebalancing
> > > > > > > > > programs).
> > > > > > > > > > > > >
> > > > > > > > > > > > > Another important consideration here is that
> someone
> > > > > managing
> > > > > > > an
> > > > > > > > > > > > > ongoing reassignment wants the most up-to-date
> > > > information,
> > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > be found on the controller.  Therefore adding this
> > > state
> > > > to
> > > > > > > > > listTopics
> > > > > > > > > > > > > or describeTopics, which could contact any node in
> the
> > > > > > > cluster, is
> > > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Finally, adding this to listTopics or
> describeTopics
> > > > feels
> > > > > > > like a
> > > > > > > > > > > warty
> > > > > > > > > > > > > API.  It's an extra boolean which interacts with
> other
> > > > > extra
> > > > > > > > > booleans
> > > > > > > > > > > > > like "show internal", etc. in weird ways.  I think
> a
> > > > > separate
> > > > > > > API
> > > > > > > > > is
> > > > > > > > > > > > > cleaner.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 3. As replicas come into sync, they will join
> the
> > > > ISR.
> > > > > > > Will we
> > > > > > > > > > > await all
> > > > > > > > > > > > > > > target replicas joining the ISR before taking
> the
> > > > > replica
> > > > > > > out
> > > > > > > > > of
> > > > > > > > > > > the target
> > > > > > > > > > > > > > > replicas set? Also, I assume that target
> replicas
> > > can
> > > > > > > still be
> > > > > > > > > > > elected as
> > > > > > > > > > > > > > > leader?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We'll take a replica out of the target replicas
> set
> > > as
> > > > > soon
> > > > > > > as
> > > > > > > > > that
> > > > > > > > > > > > > > replica is in the ISR.  Let me clarify this in
> the
> > > KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 4. Probably useful to mention permissions for
> the
> > > new
> > > > > APIs.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Good point.  I think alterPartitionAssignments
> should
> > > > > require
> > > > > > > > > ALTER
> > > > > > > > > > > on
> > > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions will be
> > > > unchanged.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I added permission information.
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Jason
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > > > > > > gwen@confluent.io>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > Looks great, and will be awesome to have
> this new
> > > > > > > capability.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe
> <
> > > > > > > > > cmccabe@apache.org>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455:
> Create
> > > an
> > > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > > Replica Reassignment.  I think this KIP is
> > > > > important
> > > > > > > since
> > > > > > > > > it
> > > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > > many follow-on improvements to Kafka
> > > reassignment
> > > > > (see
> > > > > > > the
> > > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > > section, plus a lot of the other
> discussions
> > > > we've
> > > > > had
> > > > > > > > > > > recently about
> > > > > > > > > > > > > > > > > reassignment).  It also furthers the
> important
> > > > > KIP-4
> > > > > > > goal
> > > > > > > > > of
> > > > > > > > > > > removing
> > > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I made a few changes based on the
> discussion in
> > > > the
> > > > > > > > > [DISCUSS]
> > > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > > Robert suggested, I removed the need to
> > > > explicitly
> > > > > > > cancel a
> > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > for a partition before setting up a
> different
> > > > > > > reassignment
> > > > > > > > > for
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > > specific partition.  I also simplified the
> API
> > > a
> > > > > bit by
> > > > > > > > > adding
> > > > > > > > > > > a
> > > > > > > > > > > > > > > > > PartitionReassignment class which is used
> by
> > > both
> > > > > the
> > > > > > > alter
> > > > > > > > > > > and list
> > > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I modified the proposal so that we now
> > > deprecate
> > > > > the
> > > > > > > old
> > > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > > rather than removing it completely.  That
> > > should
> > > > > give
> > > > > > > > > external
> > > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > > tools some time to transition to the new
> API.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > To clarify a question Viktor asked, I
> added a
> > > > note
> > > > > > > that the
> > > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > > > > > > --bootstrap-server
> > > > > > > > > > > argument
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > > > > > Follow us: Twitter <
> > > > https://twitter.com/ConfluentInc>
> > > > > |
> > > > > > > blog
> > > > > > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > Gwen Shapira
> > > > > > > Product Manager | Confluent
> > > > > > > 650.450.2760 | @gwenshap
> > > > > > > Follow us: Twitter | blog
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi Jun,

Thanks for taking another look at this.

On Wed, Jul 31, 2019, at 09:22, Jun Rao wrote:
> Hi, Stan,
> 
> Thanks for the explanation.
> 
> 10. If those new fields in LeaderAndIsr are only needed for future work,
> perhaps they should be added when we do the future work instead of now?

I think this ties in with one of the big goals of this KIP, making it possible to distinguish reassigning replicas from normal replicas.  This is the key to follow-on work like being able to ensure that partitions with a reassignment don't get falsely flagged as under-replicated in the metrics, or implementing reassignment quotas that don't accidentally affect normal replication traffic when a replica falls out of the ISR.

For these follow-on improvements, we need to have that information in LeaderAndIsrRequest.  We could add the information in a follow-on KIP, of course, but then all the improvements are blocked on that follow-on KIP.  That would slow things down for all of the downstream KIPs that are blocked on this.

Also, to keep things consistent, I think it would be best if the format of the data in the LeaderAndIsrRequest matched the format of the data in ZooKeeper.  Since we're deciding on the ZK format in this KIP, I think it makes sense to also decide on the format in the LeaderAndIsrRequest.

> > > Should we include those two fields in UpdateMetadata and potentially
> > > Metadata requests too?

We had some discussion earlier about how metadata responses to clients are getting too large, in part because they include a lot of information that most clients don't need (such as the ISR).  I think reassignment information definitely falls in the category of something a client doesn't need to know, so we shouldn't include it.

A program like CruiseControl, or the command-line reassignment program, just wants to get the most up-to-date information about the state of reassigning partitions.  The MetadataRequest API wouldn't deliver that, because there are inherently delays in how we propagate metadata to brokers.  That's why the ListPartitionReassignments API is a better choice for those programs.  So I think if we added this information to the MetadataResponse, nobody would actually use it, and it would just use up more bandwidth.

Of course, we can always revisit this later if we find a scenario where a producer or consumer would actually care about this.  But I think we should default to not adding stuff to the metadata response if we don't have a good use case in mind.

> > > 11. "If a new reassignment is issued during an on-going one, we cancel the
> > > current one by emptying out both AR and RR, constructing them from (the
> > > updated from the last-reassignment) R and TR, and starting anew." In this
> > > case, it seems that the controller needs to issue a StopReplica request to
> > > remove those unneeded replicas.

Good catch.  Yes, we should document this in the  KIP.

> > > 12. "Essentially, once a cancellation is called we subtract AR from R,
> > > empty out both AR and RR, and send LeaderAndIsr requests to cancel the
> > > replica movements that have not yet completed." Similar to the above, it
> > > seems the controller needs to issue a StopReplica request to remove those
> > > unneeded replicas.

Right.  Let's add this.

> > > 13. Since we changed the format of the topics/[topic] zNode, should we bump
> > > up the version number in the json value?

The change to the zNode is backwards compatible, though.  Older brokers will continue to work, but just ignore the new fields.  If we bump that version number, then downgrades will require hand-editing zookeeper.  (Of course downgrade isn't officially supported, but it would be nice not to break it if we don't need to...)  Changing the version number would also create problems during a rolling upgrade.

best,
Colin

> > >
> > > Jun
> > >
> > > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi all,
> > > >
> > > > With three non-binding +1 votes from Viktor Somogyi-Vass, Robert
> > Barrett,
> > > > and George Li, and 3 binding +1 votes from Gwen Shapira, Jason
> > Gustafson,
> > > > and myself, the vote passes.  Thanks, everyone!
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > > +1 (non-binding). Thanks for the KIP!
> > > > >
> > > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <sql_consulting@yahoo.com
> > > > .invalid>
> > > > > wrote:
> > > > >
> > > > > >  +1 (non-binding)
> > > > > >
> > > > > >
> > > > > >
> > > > > > Thanks for addressing the comments.
> > > > > > George
> > > > > >
> > > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > > > gwen@confluent.io> wrote:
> > > > > >
> > > > > >  Renewing my +1, thank you Colin and Stan for working through all
> > the
> > > > > > questions, edge cases, requests and alternatives. We ended up with
> > a
> > > > > > great protocol.
> > > > > >
> > > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> > jason@confluent.io>
> > > > > > wrote:
> > > > > > >
> > > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > > >
> > > > > > > -Jason
> > > > > > >
> > > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cmccabe@apache.org
> > >
> > > > wrote:
> > > > > > >
> > > > > > > > Thanks, Stanislav.  Let's restart the vote to reflect the fact
> > > that
> > > > > > we've
> > > > > > > > made significant changes.  The new vote will go for 3 days as
> > > > usual.
> > > > > > > >
> > > > > > > > I'll start with my +1 (binding).
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > > > > > Hey everybody,
> > > > > > > > >
> > > > > > > > > We have further iterated on the KIP in the accompanying
> > > > discussion
> > > > > > thread
> > > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > > >
> > > > > > > > > Some notable changes:
> > > > > > > > > - we will store reassignment information in the
> > > > > > `/brokers/topics/[topic]`
> > > > > > > > > - we will internally use two collections to represent a
> > > > reassignment
> > > > > > -
> > > > > > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has
> > been
> > > > > > updated
> > > > > > > > > accordingly
> > > > > > > > > - the Alter API will still use the "targetReplicas"
> > collection,
> > > > but
> > > > > > the
> > > > > > > > > List API will now return three separate collections - the
> > full
> > > > > > replica
> > > > > > > > set,
> > > > > > > > > the replicas we are adding as part of this reassignment
> > > > > > > > ("addingReplicas")
> > > > > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > > > > - cancellation of a reassignment now means a proper rollback
> > of
> > > > the
> > > > > > > > > assignment to its original state prior to the API call
> > > > > > > > >
> > > > > > > > > As always, you can re-read the KIP here
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Stanislav
> > > > > > > > >
> > > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > > cmccabe@apache.org
> > > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi George,
> > > > > > > > > >
> > > > > > > > > > Thanks for taking a look.  I am working on getting a PR
> > done
> > > > as a
> > > > > > > > > > proof-of-concept.  I'll post it soon.  Then we'll finish up
> > > the
> > > > > > vote.
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > > >  Hi Colin,
> > > > > > > > > > >
> > > > > > > > > > >  Great! Looking forward to these features.    +1
> > > > (non-binding)
> > > > > > > > > > >
> > > > > > > > > > > What is the estimated timeline to have this implemented?
> > > If
> > > > any
> > > > > > help
> > > > > > > > > > > is needed in the implementation of cancelling
> > > > reassignments,  I
> > > > > > can
> > > > > > > > > > > help if there is spare cycle.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > George
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin
> > McCabe
> > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > >
> > > > > > > > > > >  Hi George,
> > > > > > > > > > >
> > > > > > > > > > > Yes, KIP-455 allows the reassignment of individual
> > > > partitions to
> > > > > > be
> > > > > > > > > > > cancelled.  I think it's very important for these
> > > operations
> > > > to
> > > > > > be at
> > > > > > > > > > > the partition level.
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > > >  Hi Colin,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the updated KIP.  It has very good
> > > improvements
> > > > of
> > > > > > Kafka
> > > > > > > > > > > > reassignment operations.
> > > > > > > > > > > >
> > > > > > > > > > > > One question, looks like the KIP includes the
> > > Cancellation
> > > > of
> > > > > > > > > > > > individual pending reassignments as well when the
> > > > > > > > > > > > AlterPartitionReasisgnmentRequest has empty replicas
> > for
> > > > the
> > > > > > > > > > > > topic/partition. Will you also be implementing the the
> > > > > > partition
> > > > > > > > > > > > cancellation/rollback in the PR ?    If yes,  it will
> > > make
> > > > > > KIP-236
> > > > > > > > (it
> > > > > > > > > > > > has PR already) trivial, since the cancel all pending
> > > > > > > > reassignments,
> > > > > > > > > > > > one just needs to do a ListPartitionRessignmentRequest,
> > > > then
> > > > > > submit
> > > > > > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > George
> > > > > > > > > > > >
> > > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin
> > McCabe
> > > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson
> > wrote:
> > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think storing reassignment state at the partition
> > > > level
> > > > > > is
> > > > > > > > the
> > > > > > > > > > right move
> > > > > > > > > > > > > > and I also agree that replicas should understand
> > that
> > > > > > there is
> > > > > > > > a
> > > > > > > > > > > > > > reassignment in progress. This makes KIP-352 a
> > > trivial
> > > > > > > > follow-up
> > > > > > > > > > for
> > > > > > > > > > > > > > example. The only doubt I have is whether the
> > leader
> > > > and
> > > > > > isr
> > > > > > > > znode
> > > > > > > > > > is the
> > > > > > > > > > > > > > right place to store the target reassignment. It
> > is a
> > > > bit
> > > > > > odd
> > > > > > > > to
> > > > > > > > > > keep the
> > > > > > > > > > > > > > target assignment in a separate place from the
> > > current
> > > > > > > > assignment,
> > > > > > > > > > right? I
> > > > > > > > > > > > > > assume the thinking is probably that although the
> > > > current
> > > > > > > > > > assignment should
> > > > > > > > > > > > > > probably be in the leader and isr znode as well, it
> > > is
> > > > > > hard to
> > > > > > > > > > move the
> > > > > > > > > > > > > > state in a compatible way. Is that right? But if we
> > > > have no
> > > > > > > > plan
> > > > > > > > > > to remove
> > > > > > > > > > > > > > the assignment znode, do you see a downside to
> > > storing
> > > > the
> > > > > > > > target
> > > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > > >
> > > > > > > > > > > > > That's a good point -- it's probably better to keep
> > the
> > > > > > target
> > > > > > > > > > > > > assignment in the same znode as the current
> > assignment,
> > > > for
> > > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > > >
> > > > > > > > > > > > I took another look at this, and I think we should
> > stick
> > > > with
> > > > > > the
> > > > > > > > > > > > initial proposal of putting the reassignment state into
> > > > > > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > > The
> > > > > > > > reason is
> > > > > > > > > > > > because we'll want to bump the leader epoch for the
> > > > partition
> > > > > > when
> > > > > > > > > > > > changing the reassignment state, and the leader epoch
> > > > resides
> > > > > > in
> > > > > > > > that
> > > > > > > > > > > > znode anyway.  I agree there is some inconsistency
> > here,
> > > > but
> > > > > > so be
> > > > > > > > it:
> > > > > > > > > > > > if we were to greenfield these zookeeper data
> > structures,
> > > > we
> > > > > > might
> > > > > > > > do
> > > > > > > > > > > > it differently, but the proposed scheme will work fine
> > > and
> > > > be
> > > > > > > > > > > > extensible for the future.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > > It's the current assignment we're altering, right?
> > > > > > > > > > > > >
> > > > > > > > > > > > > That's fair.  AlterPartitionAssigments reads a little
> > > > > > better, and
> > > > > > > > > > I'll
> > > > > > > > > > > > > change it to that.
> > > > > > > > > > > >
> > > > > > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > > 2. Does this change affect the Metadata API? In
> > other
> > > > > > words,
> > > > > > > > are
> > > > > > > > > > clients
> > > > > > > > > > > > > > aware of reassignments? If so, then we probably
> > need
> > > a
> > > > > > change
> > > > > > > > to
> > > > > > > > > > > > > > UpdateMetadata as well. The only alternative I can
> > > > think of
> > > > > > > > would
> > > > > > > > > > be to
> > > > > > > > > > > > > > represent the replica set in the Metadata request
> > as
> > > > the
> > > > > > union
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > > current and target replicas, but I can't think of
> > any
> > > > > > benefit
> > > > > > > > to
> > > > > > > > > > hiding
> > > > > > > > > > > > > > reassignments. Note that if we did this, we
> > probably
> > > > > > wouldn't
> > > > > > > > need
> > > > > > > > > > a
> > > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I thought about this a bit... and I think on balance,
> > > > you're
> > > > > > > > right.
> > > > > > > > > > We
> > > > > > > > > > > > > should keep this information together with the
> > replica
> > > > > > nodes, isr
> > > > > > > > > > > > > nodes, and offline replicas, and that information is
> > > > > > available in
> > > > > > > > > > the
> > > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > > >  However, I do think in order to do this, we'll need
> > a
> > > > flag
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > MetadataRequest that specifiies "only show me
> > > reassigning
> > > > > > > > > > partitions".
> > > > > > > > > > > > > I'll add this.
> > > > > > > > > > > >
> > > > > > > > > > > > I revisited this, and I think we should stick with the
> > > > original
> > > > > > > > > > > > proposal of having a separate
> > ListPartitionReassignments
> > > > API.
> > > > > > > > There
> > > > > > > > > > > > really is no use case where the Producer or Consumer
> > > needs
> > > > to
> > > > > > know
> > > > > > > > > > > > about a reassignment.  They should just be notified
> > when
> > > > the
> > > > > > set of
> > > > > > > > > > > > partitions changes, which doesn't require changes to
> > > > > > > > > > > > MetadataRequest/Response.  The Admin client only cares
> > if
> > > > > > someone
> > > > > > > > is
> > > > > > > > > > > > managing the reassignment.  So adding this state to the
> > > > > > > > > > > > MetadataResponse adds overhead for no real benefit.  In
> > > the
> > > > > > common
> > > > > > > > > > case
> > > > > > > > > > > > where there is no ongoing reassignment, it would be 4
> > > > bytes per
> > > > > > > > > > > > partition of extra overhead in the MetadataResponse.
> > > > > > > > > > > >
> > > > > > > > > > > > In general, I think we have a problem of oversharing in
> > > the
> > > > > > > > > > > > MetadataRequest/Response.  As we 10x or 100x the number
> > > of
> > > > > > > > partitions
> > > > > > > > > > > > we support, we'll need to get stricter about giving
> > > clients
> > > > > > only
> > > > > > > > the
> > > > > > > > > > > > information they actually need, about the partitions
> > they
> > > > > > actually
> > > > > > > > > > care
> > > > > > > > > > > > about.  Reassignment state clearly falls in the
> > category
> > > of
> > > > > > state
> > > > > > > > that
> > > > > > > > > > > > isn't needed by clients (except very specialized
> > > > rebalancing
> > > > > > > > programs).
> > > > > > > > > > > >
> > > > > > > > > > > > Another important consideration here is that someone
> > > > managing
> > > > > > an
> > > > > > > > > > > > ongoing reassignment wants the most up-to-date
> > > information,
> > > > > > which
> > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > be found on the controller.  Therefore adding this
> > state
> > > to
> > > > > > > > listTopics
> > > > > > > > > > > > or describeTopics, which could contact any node in the
> > > > > > cluster, is
> > > > > > > > > > > > sub-optimal.
> > > > > > > > > > > >
> > > > > > > > > > > > Finally, adding this to listTopics or describeTopics
> > > feels
> > > > > > like a
> > > > > > > > > > warty
> > > > > > > > > > > > API.  It's an extra boolean which interacts with other
> > > > extra
> > > > > > > > booleans
> > > > > > > > > > > > like "show internal", etc. in weird ways.  I think a
> > > > separate
> > > > > > API
> > > > > > > > is
> > > > > > > > > > > > cleaner.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > > 3. As replicas come into sync, they will join the
> > > ISR.
> > > > > > Will we
> > > > > > > > > > await all
> > > > > > > > > > > > > > target replicas joining the ISR before taking the
> > > > replica
> > > > > > out
> > > > > > > > of
> > > > > > > > > > the target
> > > > > > > > > > > > > > replicas set? Also, I assume that target replicas
> > can
> > > > > > still be
> > > > > > > > > > elected as
> > > > > > > > > > > > > > leader?
> > > > > > > > > > > > >
> > > > > > > > > > > > > We'll take a replica out of the target replicas set
> > as
> > > > soon
> > > > > > as
> > > > > > > > that
> > > > > > > > > > > > > replica is in the ISR.  Let me clarify this in the
> > KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > > 4. Probably useful to mention permissions for the
> > new
> > > > APIs.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Good point.  I think alterPartitionAssignments should
> > > > require
> > > > > > > > ALTER
> > > > > > > > > > on
> > > > > > > > > > > > > CLUSTER.  MetadataRequest permissions will be
> > > unchanged.
> > > > > > > > > > > >
> > > > > > > > > > > > I added permission information.
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Jason
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > > > > > gwen@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > Looks great, and will be awesome to have this new
> > > > > > capability.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > > > > > > > cmccabe@apache.org>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I'd like to start the vote for KIP-455: Create
> > an
> > > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > > Replica Reassignment.  I think this KIP is
> > > > important
> > > > > > since
> > > > > > > > it
> > > > > > > > > > will unlock
> > > > > > > > > > > > > > > > many follow-on improvements to Kafka
> > reassignment
> > > > (see
> > > > > > the
> > > > > > > > > > "Future work"
> > > > > > > > > > > > > > > > section, plus a lot of the other discussions
> > > we've
> > > > had
> > > > > > > > > > recently about
> > > > > > > > > > > > > > > > reassignment).  It also furthers the important
> > > > KIP-4
> > > > > > goal
> > > > > > > > of
> > > > > > > > > > removing
> > > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I made a few changes based on the discussion in
> > > the
> > > > > > > > [DISCUSS]
> > > > > > > > > > thread.  As
> > > > > > > > > > > > > > > > Robert suggested, I removed the need to
> > > explicitly
> > > > > > cancel a
> > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > for a partition before setting up a different
> > > > > > reassignment
> > > > > > > > for
> > > > > > > > > > that
> > > > > > > > > > > > > > > > specific partition.  I also simplified the API
> > a
> > > > bit by
> > > > > > > > adding
> > > > > > > > > > a
> > > > > > > > > > > > > > > > PartitionReassignment class which is used by
> > both
> > > > the
> > > > > > alter
> > > > > > > > > > and list
> > > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I modified the proposal so that we now
> > deprecate
> > > > the
> > > > > > old
> > > > > > > > > > znode-based API
> > > > > > > > > > > > > > > > rather than removing it completely.  That
> > should
> > > > give
> > > > > > > > external
> > > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > To clarify a question Viktor asked, I added a
> > > note
> > > > > > that the
> > > > > > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > > > > > --bootstrap-server
> > > > > > > > > > argument
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > --
> > > > > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > > > > Follow us: Twitter <
> > > https://twitter.com/ConfluentInc>
> > > > |
> > > > > > blog
> > > > > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Gwen Shapira
> > > > > > Product Manager | Confluent
> > > > > > 650.450.2760 | @gwenshap
> > > > > > Follow us: Twitter | blog
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Jun Rao <ju...@confluent.io>.
Hi, Stan,

Thanks for the explanation.

10. If those new fields in LeaderAndIsr are only needed for future work,
perhaps they should be added when we do the future work instead of now?

Jun


On Wed, Jul 31, 2019 at 2:30 AM Stanislav Kozlovski <st...@confluent.io>
wrote:

> Hey Jun,
>
> I think I can answer some of your questions on behalf of Colin -- he can
> confirm if I'm correct
>
> > 10. The KIP adds two new fields (AddingReplicas and RemovingReplicas) to
> LeaderAndIsr request. Could you explain how these 2 fields will be used?
> Sorry for not explaining this in the KIP - those fields won't be used by
> the non-controller brokers yet. Our plans for them are outlined in the
> Future Work section of the KIP - namely "Reassignment Quotas that only
> throttle reassignment traffic" and "Add reassignment metrics".
>
> > Should we include those two fields in UpdateMetadata and potentially
> Metadata requests too?
> I recall this was discussed in the beginning by Colin and Jason, so I'll
> let Colin answer that question.
>
> 11 & 12. Correct, we need to send StopReplica requests. The implementation
> does this (
>
> https://github.com/apache/kafka/pull/7128/files#diff-ed90e8ecc5439a5ede5e362255d11be1R651
> )
> -- I'll update the KIP to mention it as well.
> I tried to document the algorithm here
>
> https://github.com/apache/kafka/pull/7128/files#diff-ed90e8ecc5439a5ede5e362255d11be1R521
> .
>
> 13. I think so. (
> https://github.com/apache/kafka/pull/7128#discussion_r308866206). I'll
> reflect this in the KIP
>
> Here is the updated KIP diff -
>
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=112820260&selectedPageVersions=36&selectedPageVersions=35
>
> Thanks,
> Stanislav
>
> On Tue, Jul 30, 2019 at 10:18 PM Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Colin,
> >
> > Thanks for the KIP. Sorry for the late reply. LGTM overall. A few
> detailed
> > comments below.
> >
> > 10. The KIP adds two new fields (AddingReplicas and RemovingReplicas) to
> > LeaderAndIsr request. Could you explain how these 2 fields will be used?
> > Should we include those two fields in UpdateMetadata and potentially
> > Metadata requests too?
> >
> > 11. "If a new reassignment is issued during an on-going one, we cancel
> the
> > current one by emptying out both AR and RR, constructing them from (the
> > updated from the last-reassignment) R and TR, and starting anew." In this
> > case, it seems that the controller needs to issue a StopReplica request
> to
> > remove those unneeded replicas.
> >
> > 12. "Essentially, once a cancellation is called we subtract AR from R,
> > empty out both AR and RR, and send LeaderAndIsr requests to cancel the
> > replica movements that have not yet completed." Similar to the above, it
> > seems the controller needs to issue a StopReplica request to remove those
> > unneeded replicas.
> >
> > 13. Since we changed the format of the topics/[topic] zNode, should we
> bump
> > up the version number in the json value?
> >
> > Jun
> >
> > On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi all,
> > >
> > > With three non-binding +1 votes from Viktor Somogyi-Vass, Robert
> Barrett,
> > > and George Li, and 3 binding +1 votes from Gwen Shapira, Jason
> Gustafson,
> > > and myself, the vote passes.  Thanks, everyone!
> > >
> > > best,
> > > Colin
> > >
> > > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > > +1 (non-binding). Thanks for the KIP!
> > > >
> > > > On Thu, Jul 18, 2019 at 5:59 PM George Li <sql_consulting@yahoo.com
> > > .invalid>
> > > > wrote:
> > > >
> > > > >  +1 (non-binding)
> > > > >
> > > > >
> > > > >
> > > > > Thanks for addressing the comments.
> > > > > George
> > > > >
> > > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > > gwen@confluent.io> wrote:
> > > > >
> > > > >  Renewing my +1, thank you Colin and Stan for working through all
> the
> > > > > questions, edge cases, requests and alternatives. We ended up with
> a
> > > > > great protocol.
> > > > >
> > > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <
> jason@confluent.io>
> > > > > wrote:
> > > > > >
> > > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > > >
> > > > > > -Jason
> > > > > >
> > > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cmccabe@apache.org
> >
> > > wrote:
> > > > > >
> > > > > > > Thanks, Stanislav.  Let's restart the vote to reflect the fact
> > that
> > > > > we've
> > > > > > > made significant changes.  The new vote will go for 3 days as
> > > usual.
> > > > > > >
> > > > > > > I'll start with my +1 (binding).
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > > > > Hey everybody,
> > > > > > > >
> > > > > > > > We have further iterated on the KIP in the accompanying
> > > discussion
> > > > > thread
> > > > > > > > and I'd like to propose we resume the vote.
> > > > > > > >
> > > > > > > > Some notable changes:
> > > > > > > > - we will store reassignment information in the
> > > > > `/brokers/topics/[topic]`
> > > > > > > > - we will internally use two collections to represent a
> > > reassignment
> > > > > -
> > > > > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has
> been
> > > > > updated
> > > > > > > > accordingly
> > > > > > > > - the Alter API will still use the "targetReplicas"
> collection,
> > > but
> > > > > the
> > > > > > > > List API will now return three separate collections - the
> full
> > > > > replica
> > > > > > > set,
> > > > > > > > the replicas we are adding as part of this reassignment
> > > > > > > ("addingReplicas")
> > > > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > > > - cancellation of a reassignment now means a proper rollback
> of
> > > the
> > > > > > > > assignment to its original state prior to the API call
> > > > > > > >
> > > > > > > > As always, you can re-read the KIP here
> > > > > > > >
> > > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Stanislav
> > > > > > > >
> > > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> > cmccabe@apache.org
> > > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi George,
> > > > > > > > >
> > > > > > > > > Thanks for taking a look.  I am working on getting a PR
> done
> > > as a
> > > > > > > > > proof-of-concept.  I'll post it soon.  Then we'll finish up
> > the
> > > > > vote.
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > > >  Hi Colin,
> > > > > > > > > >
> > > > > > > > > >  Great! Looking forward to these features.    +1
> > > (non-binding)
> > > > > > > > > >
> > > > > > > > > > What is the estimated timeline to have this implemented?
> > If
> > > any
> > > > > help
> > > > > > > > > > is needed in the implementation of cancelling
> > > reassignments,  I
> > > > > can
> > > > > > > > > > help if there is spare cycle.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > George
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin
> McCabe
> > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > >
> > > > > > > > > >  Hi George,
> > > > > > > > > >
> > > > > > > > > > Yes, KIP-455 allows the reassignment of individual
> > > partitions to
> > > > > be
> > > > > > > > > > cancelled.  I think it's very important for these
> > operations
> > > to
> > > > > be at
> > > > > > > > > > the partition level.
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > > >  Hi Colin,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the updated KIP.  It has very good
> > improvements
> > > of
> > > > > Kafka
> > > > > > > > > > > reassignment operations.
> > > > > > > > > > >
> > > > > > > > > > > One question, looks like the KIP includes the
> > Cancellation
> > > of
> > > > > > > > > > > individual pending reassignments as well when the
> > > > > > > > > > > AlterPartitionReasisgnmentRequest has empty replicas
> for
> > > the
> > > > > > > > > > > topic/partition. Will you also be implementing the the
> > > > > partition
> > > > > > > > > > > cancellation/rollback in the PR ?    If yes,  it will
> > make
> > > > > KIP-236
> > > > > > > (it
> > > > > > > > > > > has PR already) trivial, since the cancel all pending
> > > > > > > reassignments,
> > > > > > > > > > > one just needs to do a ListPartitionRessignmentRequest,
> > > then
> > > > > submit
> > > > > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > George
> > > > > > > > > > >
> > > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin
> McCabe
> > > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > > >
> > > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson
> wrote:
> > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think storing reassignment state at the partition
> > > level
> > > > > is
> > > > > > > the
> > > > > > > > > right move
> > > > > > > > > > > > > and I also agree that replicas should understand
> that
> > > > > there is
> > > > > > > a
> > > > > > > > > > > > > reassignment in progress. This makes KIP-352 a
> > trivial
> > > > > > > follow-up
> > > > > > > > > for
> > > > > > > > > > > > > example. The only doubt I have is whether the
> leader
> > > and
> > > > > isr
> > > > > > > znode
> > > > > > > > > is the
> > > > > > > > > > > > > right place to store the target reassignment. It
> is a
> > > bit
> > > > > odd
> > > > > > > to
> > > > > > > > > keep the
> > > > > > > > > > > > > target assignment in a separate place from the
> > current
> > > > > > > assignment,
> > > > > > > > > right? I
> > > > > > > > > > > > > assume the thinking is probably that although the
> > > current
> > > > > > > > > assignment should
> > > > > > > > > > > > > probably be in the leader and isr znode as well, it
> > is
> > > > > hard to
> > > > > > > > > move the
> > > > > > > > > > > > > state in a compatible way. Is that right? But if we
> > > have no
> > > > > > > plan
> > > > > > > > > to remove
> > > > > > > > > > > > > the assignment znode, do you see a downside to
> > storing
> > > the
> > > > > > > target
> > > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Hi Jason,
> > > > > > > > > > > >
> > > > > > > > > > > > That's a good point -- it's probably better to keep
> the
> > > > > target
> > > > > > > > > > > > assignment in the same znode as the current
> assignment,
> > > for
> > > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > > >
> > > > > > > > > > > Hi Jason,
> > > > > > > > > > >
> > > > > > > > > > > Thanks again for the review.
> > > > > > > > > > >
> > > > > > > > > > > I took another look at this, and I think we should
> stick
> > > with
> > > > > the
> > > > > > > > > > > initial proposal of putting the reassignment state into
> > > > > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > > The
> > > > > > > reason is
> > > > > > > > > > > because we'll want to bump the leader epoch for the
> > > partition
> > > > > when
> > > > > > > > > > > changing the reassignment state, and the leader epoch
> > > resides
> > > > > in
> > > > > > > that
> > > > > > > > > > > znode anyway.  I agree there is some inconsistency
> here,
> > > but
> > > > > so be
> > > > > > > it:
> > > > > > > > > > > if we were to greenfield these zookeeper data
> structures,
> > > we
> > > > > might
> > > > > > > do
> > > > > > > > > > > it differently, but the proposed scheme will work fine
> > and
> > > be
> > > > > > > > > > > extensible for the future.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > > It's the current assignment we're altering, right?
> > > > > > > > > > > >
> > > > > > > > > > > > That's fair.  AlterPartitionAssigments reads a little
> > > > > better, and
> > > > > > > > > I'll
> > > > > > > > > > > > change it to that.
> > > > > > > > > > >
> > > > > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > > 2. Does this change affect the Metadata API? In
> other
> > > > > words,
> > > > > > > are
> > > > > > > > > clients
> > > > > > > > > > > > > aware of reassignments? If so, then we probably
> need
> > a
> > > > > change
> > > > > > > to
> > > > > > > > > > > > > UpdateMetadata as well. The only alternative I can
> > > think of
> > > > > > > would
> > > > > > > > > be to
> > > > > > > > > > > > > represent the replica set in the Metadata request
> as
> > > the
> > > > > union
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > > current and target replicas, but I can't think of
> any
> > > > > benefit
> > > > > > > to
> > > > > > > > > hiding
> > > > > > > > > > > > > reassignments. Note that if we did this, we
> probably
> > > > > wouldn't
> > > > > > > need
> > > > > > > > > a
> > > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > > >
> > > > > > > > > > > > I thought about this a bit... and I think on balance,
> > > you're
> > > > > > > right.
> > > > > > > > > We
> > > > > > > > > > > > should keep this information together with the
> replica
> > > > > nodes, isr
> > > > > > > > > > > > nodes, and offline replicas, and that information is
> > > > > available in
> > > > > > > > > the
> > > > > > > > > > > > MetadataResponse.
> > > > > > > > > > > >  However, I do think in order to do this, we'll need
> a
> > > flag
> > > > > in
> > > > > > > the
> > > > > > > > > > > > MetadataRequest that specifiies "only show me
> > reassigning
> > > > > > > > > partitions".
> > > > > > > > > > > > I'll add this.
> > > > > > > > > > >
> > > > > > > > > > > I revisited this, and I think we should stick with the
> > > original
> > > > > > > > > > > proposal of having a separate
> ListPartitionReassignments
> > > API.
> > > > > > > There
> > > > > > > > > > > really is no use case where the Producer or Consumer
> > needs
> > > to
> > > > > know
> > > > > > > > > > > about a reassignment.  They should just be notified
> when
> > > the
> > > > > set of
> > > > > > > > > > > partitions changes, which doesn't require changes to
> > > > > > > > > > > MetadataRequest/Response.  The Admin client only cares
> if
> > > > > someone
> > > > > > > is
> > > > > > > > > > > managing the reassignment.  So adding this state to the
> > > > > > > > > > > MetadataResponse adds overhead for no real benefit.  In
> > the
> > > > > common
> > > > > > > > > case
> > > > > > > > > > > where there is no ongoing reassignment, it would be 4
> > > bytes per
> > > > > > > > > > > partition of extra overhead in the MetadataResponse.
> > > > > > > > > > >
> > > > > > > > > > > In general, I think we have a problem of oversharing in
> > the
> > > > > > > > > > > MetadataRequest/Response.  As we 10x or 100x the number
> > of
> > > > > > > partitions
> > > > > > > > > > > we support, we'll need to get stricter about giving
> > clients
> > > > > only
> > > > > > > the
> > > > > > > > > > > information they actually need, about the partitions
> they
> > > > > actually
> > > > > > > > > care
> > > > > > > > > > > about.  Reassignment state clearly falls in the
> category
> > of
> > > > > state
> > > > > > > that
> > > > > > > > > > > isn't needed by clients (except very specialized
> > > rebalancing
> > > > > > > programs).
> > > > > > > > > > >
> > > > > > > > > > > Another important consideration here is that someone
> > > managing
> > > > > an
> > > > > > > > > > > ongoing reassignment wants the most up-to-date
> > information,
> > > > > which
> > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > be found on the controller.  Therefore adding this
> state
> > to
> > > > > > > listTopics
> > > > > > > > > > > or describeTopics, which could contact any node in the
> > > > > cluster, is
> > > > > > > > > > > sub-optimal.
> > > > > > > > > > >
> > > > > > > > > > > Finally, adding this to listTopics or describeTopics
> > feels
> > > > > like a
> > > > > > > > > warty
> > > > > > > > > > > API.  It's an extra boolean which interacts with other
> > > extra
> > > > > > > booleans
> > > > > > > > > > > like "show internal", etc. in weird ways.  I think a
> > > separate
> > > > > API
> > > > > > > is
> > > > > > > > > > > cleaner.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > > 3. As replicas come into sync, they will join the
> > ISR.
> > > > > Will we
> > > > > > > > > await all
> > > > > > > > > > > > > target replicas joining the ISR before taking the
> > > replica
> > > > > out
> > > > > > > of
> > > > > > > > > the target
> > > > > > > > > > > > > replicas set? Also, I assume that target replicas
> can
> > > > > still be
> > > > > > > > > elected as
> > > > > > > > > > > > > leader?
> > > > > > > > > > > >
> > > > > > > > > > > > We'll take a replica out of the target replicas set
> as
> > > soon
> > > > > as
> > > > > > > that
> > > > > > > > > > > > replica is in the ISR.  Let me clarify this in the
> KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > > 4. Probably useful to mention permissions for the
> new
> > > APIs.
> > > > > > > > > > > >
> > > > > > > > > > > > Good point.  I think alterPartitionAssignments should
> > > require
> > > > > > > ALTER
> > > > > > > > > on
> > > > > > > > > > > > CLUSTER.  MetadataRequest permissions will be
> > unchanged.
> > > > > > > > > > >
> > > > > > > > > > > I added permission information.
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Jason
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > > > > gwen@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > Looks great, and will be awesome to have this new
> > > > > capability.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > > > > > > cmccabe@apache.org>
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I'd like to start the vote for KIP-455: Create
> an
> > > > > > > > > Administrative API for
> > > > > > > > > > > > > > > Replica Reassignment.  I think this KIP is
> > > important
> > > > > since
> > > > > > > it
> > > > > > > > > will unlock
> > > > > > > > > > > > > > > many follow-on improvements to Kafka
> reassignment
> > > (see
> > > > > the
> > > > > > > > > "Future work"
> > > > > > > > > > > > > > > section, plus a lot of the other discussions
> > we've
> > > had
> > > > > > > > > recently about
> > > > > > > > > > > > > > > reassignment).  It also furthers the important
> > > KIP-4
> > > > > goal
> > > > > > > of
> > > > > > > > > removing
> > > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I made a few changes based on the discussion in
> > the
> > > > > > > [DISCUSS]
> > > > > > > > > thread.  As
> > > > > > > > > > > > > > > Robert suggested, I removed the need to
> > explicitly
> > > > > cancel a
> > > > > > > > > reassignment
> > > > > > > > > > > > > > > for a partition before setting up a different
> > > > > reassignment
> > > > > > > for
> > > > > > > > > that
> > > > > > > > > > > > > > > specific partition.  I also simplified the API
> a
> > > bit by
> > > > > > > adding
> > > > > > > > > a
> > > > > > > > > > > > > > > PartitionReassignment class which is used by
> both
> > > the
> > > > > alter
> > > > > > > > > and list
> > > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I modified the proposal so that we now
> deprecate
> > > the
> > > > > old
> > > > > > > > > znode-based API
> > > > > > > > > > > > > > > rather than removing it completely.  That
> should
> > > give
> > > > > > > external
> > > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > To clarify a question Viktor asked, I added a
> > note
> > > > > that the
> > > > > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > > > > --bootstrap-server
> > > > > > > > > argument
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > --
> > > > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > > > Follow us: Twitter <
> > https://twitter.com/ConfluentInc>
> > > |
> > > > > blog
> > > > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Gwen Shapira
> > > > > Product Manager | Confluent
> > > > > 650.450.2760 | @gwenshap
> > > > > Follow us: Twitter | blog
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Stanislav Kozlovski <st...@confluent.io>.
Hey Jun,

I think I can answer some of your questions on behalf of Colin -- he can
confirm if I'm correct

> 10. The KIP adds two new fields (AddingReplicas and RemovingReplicas) to
LeaderAndIsr request. Could you explain how these 2 fields will be used?
Sorry for not explaining this in the KIP - those fields won't be used by
the non-controller brokers yet. Our plans for them are outlined in the
Future Work section of the KIP - namely "Reassignment Quotas that only
throttle reassignment traffic" and "Add reassignment metrics".

> Should we include those two fields in UpdateMetadata and potentially
Metadata requests too?
I recall this was discussed in the beginning by Colin and Jason, so I'll
let Colin answer that question.

11 & 12. Correct, we need to send StopReplica requests. The implementation
does this (
https://github.com/apache/kafka/pull/7128/files#diff-ed90e8ecc5439a5ede5e362255d11be1R651)
-- I'll update the KIP to mention it as well.
I tried to document the algorithm here
https://github.com/apache/kafka/pull/7128/files#diff-ed90e8ecc5439a5ede5e362255d11be1R521
.

13. I think so. (
https://github.com/apache/kafka/pull/7128#discussion_r308866206). I'll
reflect this in the KIP

Here is the updated KIP diff -
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=112820260&selectedPageVersions=36&selectedPageVersions=35

Thanks,
Stanislav

On Tue, Jul 30, 2019 at 10:18 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Colin,
>
> Thanks for the KIP. Sorry for the late reply. LGTM overall. A few detailed
> comments below.
>
> 10. The KIP adds two new fields (AddingReplicas and RemovingReplicas) to
> LeaderAndIsr request. Could you explain how these 2 fields will be used?
> Should we include those two fields in UpdateMetadata and potentially
> Metadata requests too?
>
> 11. "If a new reassignment is issued during an on-going one, we cancel the
> current one by emptying out both AR and RR, constructing them from (the
> updated from the last-reassignment) R and TR, and starting anew." In this
> case, it seems that the controller needs to issue a StopReplica request to
> remove those unneeded replicas.
>
> 12. "Essentially, once a cancellation is called we subtract AR from R,
> empty out both AR and RR, and send LeaderAndIsr requests to cancel the
> replica movements that have not yet completed." Similar to the above, it
> seems the controller needs to issue a StopReplica request to remove those
> unneeded replicas.
>
> 13. Since we changed the format of the topics/[topic] zNode, should we bump
> up the version number in the json value?
>
> Jun
>
> On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cm...@apache.org> wrote:
>
> > Hi all,
> >
> > With three non-binding +1 votes from Viktor Somogyi-Vass, Robert Barrett,
> > and George Li, and 3 binding +1 votes from Gwen Shapira, Jason Gustafson,
> > and myself, the vote passes.  Thanks, everyone!
> >
> > best,
> > Colin
> >
> > On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > > +1 (non-binding). Thanks for the KIP!
> > >
> > > On Thu, Jul 18, 2019 at 5:59 PM George Li <sql_consulting@yahoo.com
> > .invalid>
> > > wrote:
> > >
> > > >  +1 (non-binding)
> > > >
> > > >
> > > >
> > > > Thanks for addressing the comments.
> > > > George
> > > >
> > > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > gwen@confluent.io> wrote:
> > > >
> > > >  Renewing my +1, thank you Colin and Stan for working through all the
> > > > questions, edge cases, requests and alternatives. We ended up with a
> > > > great protocol.
> > > >
> > > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > > >
> > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > >
> > > > > > Thanks, Stanislav.  Let's restart the vote to reflect the fact
> that
> > > > we've
> > > > > > made significant changes.  The new vote will go for 3 days as
> > usual.
> > > > > >
> > > > > > I'll start with my +1 (binding).
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > >
> > > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > > > Hey everybody,
> > > > > > >
> > > > > > > We have further iterated on the KIP in the accompanying
> > discussion
> > > > thread
> > > > > > > and I'd like to propose we resume the vote.
> > > > > > >
> > > > > > > Some notable changes:
> > > > > > > - we will store reassignment information in the
> > > > `/brokers/topics/[topic]`
> > > > > > > - we will internally use two collections to represent a
> > reassignment
> > > > -
> > > > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has been
> > > > updated
> > > > > > > accordingly
> > > > > > > - the Alter API will still use the "targetReplicas" collection,
> > but
> > > > the
> > > > > > > List API will now return three separate collections - the full
> > > > replica
> > > > > > set,
> > > > > > > the replicas we are adding as part of this reassignment
> > > > > > ("addingReplicas")
> > > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > > - cancellation of a reassignment now means a proper rollback of
> > the
> > > > > > > assignment to its original state prior to the API call
> > > > > > >
> > > > > > > As always, you can re-read the KIP here
> > > > > > >
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > > >
> > > > > > > Best,
> > > > > > > Stanislav
> > > > > > >
> > > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <
> cmccabe@apache.org
> > >
> > > > wrote:
> > > > > > >
> > > > > > > > Hi George,
> > > > > > > >
> > > > > > > > Thanks for taking a look.  I am working on getting a PR done
> > as a
> > > > > > > > proof-of-concept.  I'll post it soon.  Then we'll finish up
> the
> > > > vote.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > > >  Hi Colin,
> > > > > > > > >
> > > > > > > > >  Great! Looking forward to these features.    +1
> > (non-binding)
> > > > > > > > >
> > > > > > > > > What is the estimated timeline to have this implemented?
> If
> > any
> > > > help
> > > > > > > > > is needed in the implementation of cancelling
> > reassignments,  I
> > > > can
> > > > > > > > > help if there is spare cycle.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > George
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > >
> > > > > > > > >  Hi George,
> > > > > > > > >
> > > > > > > > > Yes, KIP-455 allows the reassignment of individual
> > partitions to
> > > > be
> > > > > > > > > cancelled.  I think it's very important for these
> operations
> > to
> > > > be at
> > > > > > > > > the partition level.
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > > >  Hi Colin,
> > > > > > > > > >
> > > > > > > > > > Thanks for the updated KIP.  It has very good
> improvements
> > of
> > > > Kafka
> > > > > > > > > > reassignment operations.
> > > > > > > > > >
> > > > > > > > > > One question, looks like the KIP includes the
> Cancellation
> > of
> > > > > > > > > > individual pending reassignments as well when the
> > > > > > > > > > AlterPartitionReasisgnmentRequest has empty replicas for
> > the
> > > > > > > > > > topic/partition. Will you also be implementing the the
> > > > partition
> > > > > > > > > > cancellation/rollback in the PR ?    If yes,  it will
> make
> > > > KIP-236
> > > > > > (it
> > > > > > > > > > has PR already) trivial, since the cancel all pending
> > > > > > reassignments,
> > > > > > > > > > one just needs to do a ListPartitionRessignmentRequest,
> > then
> > > > submit
> > > > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > George
> > > > > > > > > >
> > > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > > >
> > > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > >
> > > > > > > > > > > > I think storing reassignment state at the partition
> > level
> > > > is
> > > > > > the
> > > > > > > > right move
> > > > > > > > > > > > and I also agree that replicas should understand that
> > > > there is
> > > > > > a
> > > > > > > > > > > > reassignment in progress. This makes KIP-352 a
> trivial
> > > > > > follow-up
> > > > > > > > for
> > > > > > > > > > > > example. The only doubt I have is whether the leader
> > and
> > > > isr
> > > > > > znode
> > > > > > > > is the
> > > > > > > > > > > > right place to store the target reassignment. It is a
> > bit
> > > > odd
> > > > > > to
> > > > > > > > keep the
> > > > > > > > > > > > target assignment in a separate place from the
> current
> > > > > > assignment,
> > > > > > > > right? I
> > > > > > > > > > > > assume the thinking is probably that although the
> > current
> > > > > > > > assignment should
> > > > > > > > > > > > probably be in the leader and isr znode as well, it
> is
> > > > hard to
> > > > > > > > move the
> > > > > > > > > > > > state in a compatible way. Is that right? But if we
> > have no
> > > > > > plan
> > > > > > > > to remove
> > > > > > > > > > > > the assignment znode, do you see a downside to
> storing
> > the
> > > > > > target
> > > > > > > > > > > > assignment there as well?
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Hi Jason,
> > > > > > > > > > >
> > > > > > > > > > > That's a good point -- it's probably better to keep the
> > > > target
> > > > > > > > > > > assignment in the same znode as the current assignment,
> > for
> > > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > > >
> > > > > > > > > > Hi Jason,
> > > > > > > > > >
> > > > > > > > > > Thanks again for the review.
> > > > > > > > > >
> > > > > > > > > > I took another look at this, and I think we should stick
> > with
> > > > the
> > > > > > > > > > initial proposal of putting the reassignment state into
> > > > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> > The
> > > > > > reason is
> > > > > > > > > > because we'll want to bump the leader epoch for the
> > partition
> > > > when
> > > > > > > > > > changing the reassignment state, and the leader epoch
> > resides
> > > > in
> > > > > > that
> > > > > > > > > > znode anyway.  I agree there is some inconsistency here,
> > but
> > > > so be
> > > > > > it:
> > > > > > > > > > if we were to greenfield these zookeeper data structures,
> > we
> > > > might
> > > > > > do
> > > > > > > > > > it differently, but the proposed scheme will work fine
> and
> > be
> > > > > > > > > > extensible for the future.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > A few additional questions:
> > > > > > > > > > > >
> > > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > > It's the current assignment we're altering, right?
> > > > > > > > > > >
> > > > > > > > > > > That's fair.  AlterPartitionAssigments reads a little
> > > > better, and
> > > > > > > > I'll
> > > > > > > > > > > change it to that.
> > > > > > > > > >
> > > > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > 2. Does this change affect the Metadata API? In other
> > > > words,
> > > > > > are
> > > > > > > > clients
> > > > > > > > > > > > aware of reassignments? If so, then we probably need
> a
> > > > change
> > > > > > to
> > > > > > > > > > > > UpdateMetadata as well. The only alternative I can
> > think of
> > > > > > would
> > > > > > > > be to
> > > > > > > > > > > > represent the replica set in the Metadata request as
> > the
> > > > union
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > > current and target replicas, but I can't think of any
> > > > benefit
> > > > > > to
> > > > > > > > hiding
> > > > > > > > > > > > reassignments. Note that if we did this, we probably
> > > > wouldn't
> > > > > > need
> > > > > > > > a
> > > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > > >
> > > > > > > > > > > I thought about this a bit... and I think on balance,
> > you're
> > > > > > right.
> > > > > > > > We
> > > > > > > > > > > should keep this information together with the replica
> > > > nodes, isr
> > > > > > > > > > > nodes, and offline replicas, and that information is
> > > > available in
> > > > > > > > the
> > > > > > > > > > > MetadataResponse.
> > > > > > > > > > >  However, I do think in order to do this, we'll need a
> > flag
> > > > in
> > > > > > the
> > > > > > > > > > > MetadataRequest that specifiies "only show me
> reassigning
> > > > > > > > partitions".
> > > > > > > > > > > I'll add this.
> > > > > > > > > >
> > > > > > > > > > I revisited this, and I think we should stick with the
> > original
> > > > > > > > > > proposal of having a separate ListPartitionReassignments
> > API.
> > > > > > There
> > > > > > > > > > really is no use case where the Producer or Consumer
> needs
> > to
> > > > know
> > > > > > > > > > about a reassignment.  They should just be notified when
> > the
> > > > set of
> > > > > > > > > > partitions changes, which doesn't require changes to
> > > > > > > > > > MetadataRequest/Response.  The Admin client only cares if
> > > > someone
> > > > > > is
> > > > > > > > > > managing the reassignment.  So adding this state to the
> > > > > > > > > > MetadataResponse adds overhead for no real benefit.  In
> the
> > > > common
> > > > > > > > case
> > > > > > > > > > where there is no ongoing reassignment, it would be 4
> > bytes per
> > > > > > > > > > partition of extra overhead in the MetadataResponse.
> > > > > > > > > >
> > > > > > > > > > In general, I think we have a problem of oversharing in
> the
> > > > > > > > > > MetadataRequest/Response.  As we 10x or 100x the number
> of
> > > > > > partitions
> > > > > > > > > > we support, we'll need to get stricter about giving
> clients
> > > > only
> > > > > > the
> > > > > > > > > > information they actually need, about the partitions they
> > > > actually
> > > > > > > > care
> > > > > > > > > > about.  Reassignment state clearly falls in the category
> of
> > > > state
> > > > > > that
> > > > > > > > > > isn't needed by clients (except very specialized
> > rebalancing
> > > > > > programs).
> > > > > > > > > >
> > > > > > > > > > Another important consideration here is that someone
> > managing
> > > > an
> > > > > > > > > > ongoing reassignment wants the most up-to-date
> information,
> > > > which
> > > > > > is
> > > > > > > > to
> > > > > > > > > > be found on the controller.  Therefore adding this state
> to
> > > > > > listTopics
> > > > > > > > > > or describeTopics, which could contact any node in the
> > > > cluster, is
> > > > > > > > > > sub-optimal.
> > > > > > > > > >
> > > > > > > > > > Finally, adding this to listTopics or describeTopics
> feels
> > > > like a
> > > > > > > > warty
> > > > > > > > > > API.  It's an extra boolean which interacts with other
> > extra
> > > > > > booleans
> > > > > > > > > > like "show internal", etc. in weird ways.  I think a
> > separate
> > > > API
> > > > > > is
> > > > > > > > > > cleaner.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > 3. As replicas come into sync, they will join the
> ISR.
> > > > Will we
> > > > > > > > await all
> > > > > > > > > > > > target replicas joining the ISR before taking the
> > replica
> > > > out
> > > > > > of
> > > > > > > > the target
> > > > > > > > > > > > replicas set? Also, I assume that target replicas can
> > > > still be
> > > > > > > > elected as
> > > > > > > > > > > > leader?
> > > > > > > > > > >
> > > > > > > > > > > We'll take a replica out of the target replicas set as
> > soon
> > > > as
> > > > > > that
> > > > > > > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > > > > > > >
> > > > > > > > > > > > 4. Probably useful to mention permissions for the new
> > APIs.
> > > > > > > > > > >
> > > > > > > > > > > Good point.  I think alterPartitionAssignments should
> > require
> > > > > > ALTER
> > > > > > > > on
> > > > > > > > > > > CLUSTER.  MetadataRequest permissions will be
> unchanged.
> > > > > > > > > >
> > > > > > > > > > I added permission information.
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Jason
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > > > gwen@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > Looks great, and will be awesome to have this new
> > > > capability.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > > > > > cmccabe@apache.org>
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > > > > > > Administrative API for
> > > > > > > > > > > > > > Replica Reassignment.  I think this KIP is
> > important
> > > > since
> > > > > > it
> > > > > > > > will unlock
> > > > > > > > > > > > > > many follow-on improvements to Kafka reassignment
> > (see
> > > > the
> > > > > > > > "Future work"
> > > > > > > > > > > > > > section, plus a lot of the other discussions
> we've
> > had
> > > > > > > > recently about
> > > > > > > > > > > > > > reassignment).  It also furthers the important
> > KIP-4
> > > > goal
> > > > > > of
> > > > > > > > removing
> > > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I made a few changes based on the discussion in
> the
> > > > > > [DISCUSS]
> > > > > > > > thread.  As
> > > > > > > > > > > > > > Robert suggested, I removed the need to
> explicitly
> > > > cancel a
> > > > > > > > reassignment
> > > > > > > > > > > > > > for a partition before setting up a different
> > > > reassignment
> > > > > > for
> > > > > > > > that
> > > > > > > > > > > > > > specific partition.  I also simplified the API a
> > bit by
> > > > > > adding
> > > > > > > > a
> > > > > > > > > > > > > > PartitionReassignment class which is used by both
> > the
> > > > alter
> > > > > > > > and list
> > > > > > > > > > > > > APIs.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I modified the proposal so that we now deprecate
> > the
> > > > old
> > > > > > > > znode-based API
> > > > > > > > > > > > > > rather than removing it completely.  That should
> > give
> > > > > > external
> > > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > To clarify a question Viktor asked, I added a
> note
> > > > that the
> > > > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > > > --bootstrap-server
> > > > > > > > argument
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > --
> > > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > > Follow us: Twitter <
> https://twitter.com/ConfluentInc>
> > |
> > > > blog
> > > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Gwen Shapira
> > > > Product Manager | Confluent
> > > > 650.450.2760 | @gwenshap
> > > > Follow us: Twitter | blog
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Jun Rao <ju...@confluent.io>.
Hi, Colin,

Thanks for the KIP. Sorry for the late reply. LGTM overall. A few detailed
comments below.

10. The KIP adds two new fields (AddingReplicas and RemovingReplicas) to
LeaderAndIsr request. Could you explain how these 2 fields will be used?
Should we include those two fields in UpdateMetadata and potentially
Metadata requests too?

11. "If a new reassignment is issued during an on-going one, we cancel the
current one by emptying out both AR and RR, constructing them from (the
updated from the last-reassignment) R and TR, and starting anew." In this
case, it seems that the controller needs to issue a StopReplica request to
remove those unneeded replicas.

12. "Essentially, once a cancellation is called we subtract AR from R,
empty out both AR and RR, and send LeaderAndIsr requests to cancel the
replica movements that have not yet completed." Similar to the above, it
seems the controller needs to issue a StopReplica request to remove those
unneeded replicas.

13. Since we changed the format of the topics/[topic] zNode, should we bump
up the version number in the json value?

Jun

On Mon, Jul 22, 2019 at 8:38 AM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> With three non-binding +1 votes from Viktor Somogyi-Vass, Robert Barrett,
> and George Li, and 3 binding +1 votes from Gwen Shapira, Jason Gustafson,
> and myself, the vote passes.  Thanks, everyone!
>
> best,
> Colin
>
> On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> > +1 (non-binding). Thanks for the KIP!
> >
> > On Thu, Jul 18, 2019 at 5:59 PM George Li <sql_consulting@yahoo.com
> .invalid>
> > wrote:
> >
> > >  +1 (non-binding)
> > >
> > >
> > >
> > > Thanks for addressing the comments.
> > > George
> > >
> > >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > gwen@confluent.io> wrote:
> > >
> > >  Renewing my +1, thank you Colin and Stan for working through all the
> > > questions, edge cases, requests and alternatives. We ended up with a
> > > great protocol.
> > >
> > > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > > >
> > > > +1 Thanks for the KIP. Really looking forward to this!
> > > >
> > > > -Jason
> > > >
> > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > >
> > > > > Thanks, Stanislav.  Let's restart the vote to reflect the fact that
> > > we've
> > > > > made significant changes.  The new vote will go for 3 days as
> usual.
> > > > >
> > > > > I'll start with my +1 (binding).
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > > Hey everybody,
> > > > > >
> > > > > > We have further iterated on the KIP in the accompanying
> discussion
> > > thread
> > > > > > and I'd like to propose we resume the vote.
> > > > > >
> > > > > > Some notable changes:
> > > > > > - we will store reassignment information in the
> > > `/brokers/topics/[topic]`
> > > > > > - we will internally use two collections to represent a
> reassignment
> > > -
> > > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has been
> > > updated
> > > > > > accordingly
> > > > > > - the Alter API will still use the "targetReplicas" collection,
> but
> > > the
> > > > > > List API will now return three separate collections - the full
> > > replica
> > > > > set,
> > > > > > the replicas we are adding as part of this reassignment
> > > > > ("addingReplicas")
> > > > > > and the replicas we are removing ("removingReplicas")
> > > > > > - cancellation of a reassignment now means a proper rollback of
> the
> > > > > > assignment to its original state prior to the API call
> > > > > >
> > > > > > As always, you can re-read the KIP here
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > > >
> > > > > > Best,
> > > > > > Stanislav
> > > > > >
> > > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cmccabe@apache.org
> >
> > > wrote:
> > > > > >
> > > > > > > Hi George,
> > > > > > >
> > > > > > > Thanks for taking a look.  I am working on getting a PR done
> as a
> > > > > > > proof-of-concept.  I'll post it soon.  Then we'll finish up the
> > > vote.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > > >  Hi Colin,
> > > > > > > >
> > > > > > > >  Great! Looking forward to these features.    +1
> (non-binding)
> > > > > > > >
> > > > > > > > What is the estimated timeline to have this implemented?  If
> any
> > > help
> > > > > > > > is needed in the implementation of cancelling
> reassignments,  I
> > > can
> > > > > > > > help if there is spare cycle.
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > George
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > > > > > <cm...@apache.org> wrote:
> > > > > > > >
> > > > > > > >  Hi George,
> > > > > > > >
> > > > > > > > Yes, KIP-455 allows the reassignment of individual
> partitions to
> > > be
> > > > > > > > cancelled.  I think it's very important for these operations
> to
> > > be at
> > > > > > > > the partition level.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > > >  Hi Colin,
> > > > > > > > >
> > > > > > > > > Thanks for the updated KIP.  It has very good improvements
> of
> > > Kafka
> > > > > > > > > reassignment operations.
> > > > > > > > >
> > > > > > > > > One question, looks like the KIP includes the Cancellation
> of
> > > > > > > > > individual pending reassignments as well when the
> > > > > > > > > AlterPartitionReasisgnmentRequest has empty replicas for
> the
> > > > > > > > > topic/partition. Will you also be implementing the the
> > > partition
> > > > > > > > > cancellation/rollback in the PR ?    If yes,  it will make
> > > KIP-236
> > > > > (it
> > > > > > > > > has PR already) trivial, since the cancel all pending
> > > > > reassignments,
> > > > > > > > > one just needs to do a ListPartitionRessignmentRequest,
> then
> > > submit
> > > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > George
> > > > > > > > >
> > > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > > > > > <cm...@apache.org> wrote:
> > > > > > > > >
> > > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > > > > > Hi Colin,
> > > > > > > > > > >
> > > > > > > > > > > I think storing reassignment state at the partition
> level
> > > is
> > > > > the
> > > > > > > right move
> > > > > > > > > > > and I also agree that replicas should understand that
> > > there is
> > > > > a
> > > > > > > > > > > reassignment in progress. This makes KIP-352 a trivial
> > > > > follow-up
> > > > > > > for
> > > > > > > > > > > example. The only doubt I have is whether the leader
> and
> > > isr
> > > > > znode
> > > > > > > is the
> > > > > > > > > > > right place to store the target reassignment. It is a
> bit
> > > odd
> > > > > to
> > > > > > > keep the
> > > > > > > > > > > target assignment in a separate place from the current
> > > > > assignment,
> > > > > > > right? I
> > > > > > > > > > > assume the thinking is probably that although the
> current
> > > > > > > assignment should
> > > > > > > > > > > probably be in the leader and isr znode as well, it is
> > > hard to
> > > > > > > move the
> > > > > > > > > > > state in a compatible way. Is that right? But if we
> have no
> > > > > plan
> > > > > > > to remove
> > > > > > > > > > > the assignment znode, do you see a downside to storing
> the
> > > > > target
> > > > > > > > > > > assignment there as well?
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Hi Jason,
> > > > > > > > > >
> > > > > > > > > > That's a good point -- it's probably better to keep the
> > > target
> > > > > > > > > > assignment in the same znode as the current assignment,
> for
> > > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > > >
> > > > > > > > > Hi Jason,
> > > > > > > > >
> > > > > > > > > Thanks again for the review.
> > > > > > > > >
> > > > > > > > > I took another look at this, and I think we should stick
> with
> > > the
> > > > > > > > > initial proposal of putting the reassignment state into
> > > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.
> The
> > > > > reason is
> > > > > > > > > because we'll want to bump the leader epoch for the
> partition
> > > when
> > > > > > > > > changing the reassignment state, and the leader epoch
> resides
> > > in
> > > > > that
> > > > > > > > > znode anyway.  I agree there is some inconsistency here,
> but
> > > so be
> > > > > it:
> > > > > > > > > if we were to greenfield these zookeeper data structures,
> we
> > > might
> > > > > do
> > > > > > > > > it differently, but the proposed scheme will work fine and
> be
> > > > > > > > > extensible for the future.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > A few additional questions:
> > > > > > > > > > >
> > > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > > `alterPartitionAssignments`?
> > > > > > > > > > > It's the current assignment we're altering, right?
> > > > > > > > > >
> > > > > > > > > > That's fair.  AlterPartitionAssigments reads a little
> > > better, and
> > > > > > > I'll
> > > > > > > > > > change it to that.
> > > > > > > > >
> > > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > 2. Does this change affect the Metadata API? In other
> > > words,
> > > > > are
> > > > > > > clients
> > > > > > > > > > > aware of reassignments? If so, then we probably need a
> > > change
> > > > > to
> > > > > > > > > > > UpdateMetadata as well. The only alternative I can
> think of
> > > > > would
> > > > > > > be to
> > > > > > > > > > > represent the replica set in the Metadata request as
> the
> > > union
> > > > > of
> > > > > > > the
> > > > > > > > > > > current and target replicas, but I can't think of any
> > > benefit
> > > > > to
> > > > > > > hiding
> > > > > > > > > > > reassignments. Note that if we did this, we probably
> > > wouldn't
> > > > > need
> > > > > > > a
> > > > > > > > > > > separate API to list reassignments.
> > > > > > > > > >
> > > > > > > > > > I thought about this a bit... and I think on balance,
> you're
> > > > > right.
> > > > > > > We
> > > > > > > > > > should keep this information together with the replica
> > > nodes, isr
> > > > > > > > > > nodes, and offline replicas, and that information is
> > > available in
> > > > > > > the
> > > > > > > > > > MetadataResponse.
> > > > > > > > > >  However, I do think in order to do this, we'll need a
> flag
> > > in
> > > > > the
> > > > > > > > > > MetadataRequest that specifiies "only show me reassigning
> > > > > > > partitions".
> > > > > > > > > > I'll add this.
> > > > > > > > >
> > > > > > > > > I revisited this, and I think we should stick with the
> original
> > > > > > > > > proposal of having a separate ListPartitionReassignments
> API.
> > > > > There
> > > > > > > > > really is no use case where the Producer or Consumer needs
> to
> > > know
> > > > > > > > > about a reassignment.  They should just be notified when
> the
> > > set of
> > > > > > > > > partitions changes, which doesn't require changes to
> > > > > > > > > MetadataRequest/Response.  The Admin client only cares if
> > > someone
> > > > > is
> > > > > > > > > managing the reassignment.  So adding this state to the
> > > > > > > > > MetadataResponse adds overhead for no real benefit.  In the
> > > common
> > > > > > > case
> > > > > > > > > where there is no ongoing reassignment, it would be 4
> bytes per
> > > > > > > > > partition of extra overhead in the MetadataResponse.
> > > > > > > > >
> > > > > > > > > In general, I think we have a problem of oversharing in the
> > > > > > > > > MetadataRequest/Response.  As we 10x or 100x the number of
> > > > > partitions
> > > > > > > > > we support, we'll need to get stricter about giving clients
> > > only
> > > > > the
> > > > > > > > > information they actually need, about the partitions they
> > > actually
> > > > > > > care
> > > > > > > > > about.  Reassignment state clearly falls in the category of
> > > state
> > > > > that
> > > > > > > > > isn't needed by clients (except very specialized
> rebalancing
> > > > > programs).
> > > > > > > > >
> > > > > > > > > Another important consideration here is that someone
> managing
> > > an
> > > > > > > > > ongoing reassignment wants the most up-to-date information,
> > > which
> > > > > is
> > > > > > > to
> > > > > > > > > be found on the controller.  Therefore adding this state to
> > > > > listTopics
> > > > > > > > > or describeTopics, which could contact any node in the
> > > cluster, is
> > > > > > > > > sub-optimal.
> > > > > > > > >
> > > > > > > > > Finally, adding this to listTopics or describeTopics feels
> > > like a
> > > > > > > warty
> > > > > > > > > API.  It's an extra boolean which interacts with other
> extra
> > > > > booleans
> > > > > > > > > like "show internal", etc. in weird ways.  I think a
> separate
> > > API
> > > > > is
> > > > > > > > > cleaner.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > 3. As replicas come into sync, they will join the ISR.
> > > Will we
> > > > > > > await all
> > > > > > > > > > > target replicas joining the ISR before taking the
> replica
> > > out
> > > > > of
> > > > > > > the target
> > > > > > > > > > > replicas set? Also, I assume that target replicas can
> > > still be
> > > > > > > elected as
> > > > > > > > > > > leader?
> > > > > > > > > >
> > > > > > > > > > We'll take a replica out of the target replicas set as
> soon
> > > as
> > > > > that
> > > > > > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > > > > > >
> > > > > > > > > > > 4. Probably useful to mention permissions for the new
> APIs.
> > > > > > > > > >
> > > > > > > > > > Good point.  I think alterPartitionAssignments should
> require
> > > > > ALTER
> > > > > > > on
> > > > > > > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > > > > > > >
> > > > > > > > > I added permission information.
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Jason
> > > > > > > > > > >
> > > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > > gwen@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > Looks great, and will be awesome to have this new
> > > capability.
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > > > > cmccabe@apache.org>
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > > > > > Administrative API for
> > > > > > > > > > > > > Replica Reassignment.  I think this KIP is
> important
> > > since
> > > > > it
> > > > > > > will unlock
> > > > > > > > > > > > > many follow-on improvements to Kafka reassignment
> (see
> > > the
> > > > > > > "Future work"
> > > > > > > > > > > > > section, plus a lot of the other discussions we've
> had
> > > > > > > recently about
> > > > > > > > > > > > > reassignment).  It also furthers the important
> KIP-4
> > > goal
> > > > > of
> > > > > > > removing
> > > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I made a few changes based on the discussion in the
> > > > > [DISCUSS]
> > > > > > > thread.  As
> > > > > > > > > > > > > Robert suggested, I removed the need to explicitly
> > > cancel a
> > > > > > > reassignment
> > > > > > > > > > > > > for a partition before setting up a different
> > > reassignment
> > > > > for
> > > > > > > that
> > > > > > > > > > > > > specific partition.  I also simplified the API a
> bit by
> > > > > adding
> > > > > > > a
> > > > > > > > > > > > > PartitionReassignment class which is used by both
> the
> > > alter
> > > > > > > and list
> > > > > > > > > > > > APIs.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I modified the proposal so that we now deprecate
> the
> > > old
> > > > > > > znode-based API
> > > > > > > > > > > > > rather than removing it completely.  That should
> give
> > > > > external
> > > > > > > > > > > > rebalancing
> > > > > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > > > > >
> > > > > > > > > > > > > To clarify a question Viktor asked, I added a note
> > > that the
> > > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > > --bootstrap-server
> > > > > > > argument
> > > > > > > > > > > > to
> > > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > > >
> > > > > > > > > > > > > thanks,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > --
> > > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc>
> |
> > > blog
> > > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> > >
> > >
> > > --
> > > Gwen Shapira
> > > Product Manager | Confluent
> > > 650.450.2760 | @gwenshap
> > > Follow us: Twitter | blog
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi all,

With three non-binding +1 votes from Viktor Somogyi-Vass, Robert Barrett, and George Li, and 3 binding +1 votes from Gwen Shapira, Jason Gustafson, and myself, the vote passes.  Thanks, everyone!

best,
Colin

On Fri, Jul 19, 2019, at 08:55, Robert Barrett wrote:
> +1 (non-binding). Thanks for the KIP!
> 
> On Thu, Jul 18, 2019 at 5:59 PM George Li <sq...@yahoo.com.invalid>
> wrote:
> 
> >  +1 (non-binding)
> >
> >
> >
> > Thanks for addressing the comments.
> > George
> >
> >     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > gwen@confluent.io> wrote:
> >
> >  Renewing my +1, thank you Colin and Stan for working through all the
> > questions, edge cases, requests and alternatives. We ended up with a
> > great protocol.
> >
> > On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <ja...@confluent.io>
> > wrote:
> > >
> > > +1 Thanks for the KIP. Really looking forward to this!
> > >
> > > -Jason
> > >
> > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Thanks, Stanislav.  Let's restart the vote to reflect the fact that
> > we've
> > > > made significant changes.  The new vote will go for 3 days as usual.
> > > >
> > > > I'll start with my +1 (binding).
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > > Hey everybody,
> > > > >
> > > > > We have further iterated on the KIP in the accompanying discussion
> > thread
> > > > > and I'd like to propose we resume the vote.
> > > > >
> > > > > Some notable changes:
> > > > > - we will store reassignment information in the
> > `/brokers/topics/[topic]`
> > > > > - we will internally use two collections to represent a reassignment
> > -
> > > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has been
> > updated
> > > > > accordingly
> > > > > - the Alter API will still use the "targetReplicas" collection, but
> > the
> > > > > List API will now return three separate collections - the full
> > replica
> > > > set,
> > > > > the replicas we are adding as part of this reassignment
> > > > ("addingReplicas")
> > > > > and the replicas we are removing ("removingReplicas")
> > > > > - cancellation of a reassignment now means a proper rollback of the
> > > > > assignment to its original state prior to the API call
> > > > >
> > > > > As always, you can re-read the KIP here
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > > >
> > > > > Best,
> > > > > Stanislav
> > > > >
> > > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > >
> > > > > > Hi George,
> > > > > >
> > > > > > Thanks for taking a look.  I am working on getting a PR done as a
> > > > > > proof-of-concept.  I'll post it soon.  Then we'll finish up the
> > vote.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > > >  Hi Colin,
> > > > > > >
> > > > > > >  Great! Looking forward to these features.    +1 (non-binding)
> > > > > > >
> > > > > > > What is the estimated timeline to have this implemented?  If any
> > help
> > > > > > > is needed in the implementation of cancelling reassignments,  I
> > can
> > > > > > > help if there is spare cycle.
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > George
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > > > > <cm...@apache.org> wrote:
> > > > > > >
> > > > > > >  Hi George,
> > > > > > >
> > > > > > > Yes, KIP-455 allows the reassignment of individual partitions to
> > be
> > > > > > > cancelled.  I think it's very important for these operations to
> > be at
> > > > > > > the partition level.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > > >  Hi Colin,
> > > > > > > >
> > > > > > > > Thanks for the updated KIP.  It has very good improvements of
> > Kafka
> > > > > > > > reassignment operations.
> > > > > > > >
> > > > > > > > One question, looks like the KIP includes the Cancellation of
> > > > > > > > individual pending reassignments as well when the
> > > > > > > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > > > > > > topic/partition. Will you also be implementing the the
> > partition
> > > > > > > > cancellation/rollback in the PR ?    If yes,  it will make
> > KIP-236
> > > > (it
> > > > > > > > has PR already) trivial, since the cancel all pending
> > > > reassignments,
> > > > > > > > one just needs to do a ListPartitionRessignmentRequest, then
> > submit
> > > > > > > > empty replicas for all those topic/partitions in
> > > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > George
> > > > > > > >
> > > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > > > > <cm...@apache.org> wrote:
> > > > > > > >
> > > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > > > > Hi Colin,
> > > > > > > > > >
> > > > > > > > > > I think storing reassignment state at the partition level
> > is
> > > > the
> > > > > > right move
> > > > > > > > > > and I also agree that replicas should understand that
> > there is
> > > > a
> > > > > > > > > > reassignment in progress. This makes KIP-352 a trivial
> > > > follow-up
> > > > > > for
> > > > > > > > > > example. The only doubt I have is whether the leader and
> > isr
> > > > znode
> > > > > > is the
> > > > > > > > > > right place to store the target reassignment. It is a bit
> > odd
> > > > to
> > > > > > keep the
> > > > > > > > > > target assignment in a separate place from the current
> > > > assignment,
> > > > > > right? I
> > > > > > > > > > assume the thinking is probably that although the current
> > > > > > assignment should
> > > > > > > > > > probably be in the leader and isr znode as well, it is
> > hard to
> > > > > > move the
> > > > > > > > > > state in a compatible way. Is that right? But if we have no
> > > > plan
> > > > > > to remove
> > > > > > > > > > the assignment znode, do you see a downside to storing the
> > > > target
> > > > > > > > > > assignment there as well?
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > Hi Jason,
> > > > > > > > >
> > > > > > > > > That's a good point -- it's probably better to keep the
> > target
> > > > > > > > > assignment in the same znode as the current assignment, for
> > > > > > > > > consistency.  I'll change the KIP.
> > > > > > > >
> > > > > > > > Hi Jason,
> > > > > > > >
> > > > > > > > Thanks again for the review.
> > > > > > > >
> > > > > > > > I took another look at this, and I think we should stick with
> > the
> > > > > > > > initial proposal of putting the reassignment state into
> > > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The
> > > > reason is
> > > > > > > > because we'll want to bump the leader epoch for the partition
> > when
> > > > > > > > changing the reassignment state, and the leader epoch resides
> > in
> > > > that
> > > > > > > > znode anyway.  I agree there is some inconsistency here, but
> > so be
> > > > it:
> > > > > > > > if we were to greenfield these zookeeper data structures, we
> > might
> > > > do
> > > > > > > > it differently, but the proposed scheme will work fine and be
> > > > > > > > extensible for the future.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > > A few additional questions:
> > > > > > > > > >
> > > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > > `alterPartitionAssignments`?
> > > > > > > > > > It's the current assignment we're altering, right?
> > > > > > > > >
> > > > > > > > > That's fair.  AlterPartitionAssigments reads a little
> > better, and
> > > > > > I'll
> > > > > > > > > change it to that.
> > > > > > > >
> > > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > > 2. Does this change affect the Metadata API? In other
> > words,
> > > > are
> > > > > > clients
> > > > > > > > > > aware of reassignments? If so, then we probably need a
> > change
> > > > to
> > > > > > > > > > UpdateMetadata as well. The only alternative I can think of
> > > > would
> > > > > > be to
> > > > > > > > > > represent the replica set in the Metadata request as the
> > union
> > > > of
> > > > > > the
> > > > > > > > > > current and target replicas, but I can't think of any
> > benefit
> > > > to
> > > > > > hiding
> > > > > > > > > > reassignments. Note that if we did this, we probably
> > wouldn't
> > > > need
> > > > > > a
> > > > > > > > > > separate API to list reassignments.
> > > > > > > > >
> > > > > > > > > I thought about this a bit... and I think on balance, you're
> > > > right.
> > > > > > We
> > > > > > > > > should keep this information together with the replica
> > nodes, isr
> > > > > > > > > nodes, and offline replicas, and that information is
> > available in
> > > > > > the
> > > > > > > > > MetadataResponse.
> > > > > > > > >  However, I do think in order to do this, we'll need a flag
> > in
> > > > the
> > > > > > > > > MetadataRequest that specifiies "only show me reassigning
> > > > > > partitions".
> > > > > > > > > I'll add this.
> > > > > > > >
> > > > > > > > I revisited this, and I think we should stick with the original
> > > > > > > > proposal of having a separate ListPartitionReassignments API.
> > > > There
> > > > > > > > really is no use case where the Producer or Consumer needs to
> > know
> > > > > > > > about a reassignment.  They should just be notified when the
> > set of
> > > > > > > > partitions changes, which doesn't require changes to
> > > > > > > > MetadataRequest/Response.  The Admin client only cares if
> > someone
> > > > is
> > > > > > > > managing the reassignment.  So adding this state to the
> > > > > > > > MetadataResponse adds overhead for no real benefit.  In the
> > common
> > > > > > case
> > > > > > > > where there is no ongoing reassignment, it would be 4 bytes per
> > > > > > > > partition of extra overhead in the MetadataResponse.
> > > > > > > >
> > > > > > > > In general, I think we have a problem of oversharing in the
> > > > > > > > MetadataRequest/Response.  As we 10x or 100x the number of
> > > > partitions
> > > > > > > > we support, we'll need to get stricter about giving clients
> > only
> > > > the
> > > > > > > > information they actually need, about the partitions they
> > actually
> > > > > > care
> > > > > > > > about.  Reassignment state clearly falls in the category of
> > state
> > > > that
> > > > > > > > isn't needed by clients (except very specialized rebalancing
> > > > programs).
> > > > > > > >
> > > > > > > > Another important consideration here is that someone managing
> > an
> > > > > > > > ongoing reassignment wants the most up-to-date information,
> > which
> > > > is
> > > > > > to
> > > > > > > > be found on the controller.  Therefore adding this state to
> > > > listTopics
> > > > > > > > or describeTopics, which could contact any node in the
> > cluster, is
> > > > > > > > sub-optimal.
> > > > > > > >
> > > > > > > > Finally, adding this to listTopics or describeTopics feels
> > like a
> > > > > > warty
> > > > > > > > API.  It's an extra boolean which interacts with other extra
> > > > booleans
> > > > > > > > like "show internal", etc. in weird ways.  I think a separate
> > API
> > > > is
> > > > > > > > cleaner.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > > 3. As replicas come into sync, they will join the ISR.
> > Will we
> > > > > > await all
> > > > > > > > > > target replicas joining the ISR before taking the replica
> > out
> > > > of
> > > > > > the target
> > > > > > > > > > replicas set? Also, I assume that target replicas can
> > still be
> > > > > > elected as
> > > > > > > > > > leader?
> > > > > > > > >
> > > > > > > > > We'll take a replica out of the target replicas set as soon
> > as
> > > > that
> > > > > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > > > > >
> > > > > > > > > > 4. Probably useful to mention permissions for the new APIs.
> > > > > > > > >
> > > > > > > > > Good point.  I think alterPartitionAssignments should require
> > > > ALTER
> > > > > > on
> > > > > > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > > > > > >
> > > > > > > > I added permission information.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Jason
> > > > > > > > > >
> > > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > > gwen@confluent.io>
> > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 (binding)
> > > > > > > > > > > Looks great, and will be awesome to have this new
> > capability.
> > > > > > > > > > >
> > > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > > > cmccabe@apache.org>
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi all,
> > > > > > > > > > > >
> > > > > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > > > > Administrative API for
> > > > > > > > > > > > Replica Reassignment.  I think this KIP is important
> > since
> > > > it
> > > > > > will unlock
> > > > > > > > > > > > many follow-on improvements to Kafka reassignment (see
> > the
> > > > > > "Future work"
> > > > > > > > > > > > section, plus a lot of the other discussions we've had
> > > > > > recently about
> > > > > > > > > > > > reassignment).  It also furthers the important KIP-4
> > goal
> > > > of
> > > > > > removing
> > > > > > > > > > > > direct access to ZK.
> > > > > > > > > > > >
> > > > > > > > > > > > I made a few changes based on the discussion in the
> > > > [DISCUSS]
> > > > > > thread.  As
> > > > > > > > > > > > Robert suggested, I removed the need to explicitly
> > cancel a
> > > > > > reassignment
> > > > > > > > > > > > for a partition before setting up a different
> > reassignment
> > > > for
> > > > > > that
> > > > > > > > > > > > specific partition.  I also simplified the API a bit by
> > > > adding
> > > > > > a
> > > > > > > > > > > > PartitionReassignment class which is used by both the
> > alter
> > > > > > and list
> > > > > > > > > > > APIs.
> > > > > > > > > > > >
> > > > > > > > > > > > I modified the proposal so that we now deprecate the
> > old
> > > > > > znode-based API
> > > > > > > > > > > > rather than removing it completely.  That should give
> > > > external
> > > > > > > > > > > rebalancing
> > > > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > > > >
> > > > > > > > > > > > To clarify a question Viktor asked, I added a note
> > that the
> > > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > > --bootstrap-server
> > > > > > argument
> > > > > > > > > > > to
> > > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > > >
> > > > > > > > > > > > thanks,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > --
> > > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> > blog
> > > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> >
> >
> >
> > --
> > Gwen Shapira
> > Product Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter | blog
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Robert Barrett <bo...@confluent.io>.
+1 (non-binding). Thanks for the KIP!

On Thu, Jul 18, 2019 at 5:59 PM George Li <sq...@yahoo.com.invalid>
wrote:

>  +1 (non-binding)
>
>
>
> Thanks for addressing the comments.
> George
>
>     On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> gwen@confluent.io> wrote:
>
>  Renewing my +1, thank you Colin and Stan for working through all the
> questions, edge cases, requests and alternatives. We ended up with a
> great protocol.
>
> On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <ja...@confluent.io>
> wrote:
> >
> > +1 Thanks for the KIP. Really looking forward to this!
> >
> > -Jason
> >
> > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Thanks, Stanislav.  Let's restart the vote to reflect the fact that
> we've
> > > made significant changes.  The new vote will go for 3 days as usual.
> > >
> > > I'll start with my +1 (binding).
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > > Hey everybody,
> > > >
> > > > We have further iterated on the KIP in the accompanying discussion
> thread
> > > > and I'd like to propose we resume the vote.
> > > >
> > > > Some notable changes:
> > > > - we will store reassignment information in the
> `/brokers/topics/[topic]`
> > > > - we will internally use two collections to represent a reassignment
> -
> > > > "addingReplicas" and "removingReplicas". LeaderAndIsr has been
> updated
> > > > accordingly
> > > > - the Alter API will still use the "targetReplicas" collection, but
> the
> > > > List API will now return three separate collections - the full
> replica
> > > set,
> > > > the replicas we are adding as part of this reassignment
> > > ("addingReplicas")
> > > > and the replicas we are removing ("removingReplicas")
> > > > - cancellation of a reassignment now means a proper rollback of the
> > > > assignment to its original state prior to the API call
> > > >
> > > > As always, you can re-read the KIP here
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > >
> > > > Best,
> > > > Stanislav
> > > >
> > > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > >
> > > > > Hi George,
> > > > >
> > > > > Thanks for taking a look.  I am working on getting a PR done as a
> > > > > proof-of-concept.  I'll post it soon.  Then we'll finish up the
> vote.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > > >  Hi Colin,
> > > > > >
> > > > > >  Great! Looking forward to these features.    +1 (non-binding)
> > > > > >
> > > > > > What is the estimated timeline to have this implemented?  If any
> help
> > > > > > is needed in the implementation of cancelling reassignments,  I
> can
> > > > > > help if there is spare cycle.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > George
> > > > > >
> > > > > >
> > > > > >
> > > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > > > <cm...@apache.org> wrote:
> > > > > >
> > > > > >  Hi George,
> > > > > >
> > > > > > Yes, KIP-455 allows the reassignment of individual partitions to
> be
> > > > > > cancelled.  I think it's very important for these operations to
> be at
> > > > > > the partition level.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > > >  Hi Colin,
> > > > > > >
> > > > > > > Thanks for the updated KIP.  It has very good improvements of
> Kafka
> > > > > > > reassignment operations.
> > > > > > >
> > > > > > > One question, looks like the KIP includes the Cancellation of
> > > > > > > individual pending reassignments as well when the
> > > > > > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > > > > > topic/partition. Will you also be implementing the the
> partition
> > > > > > > cancellation/rollback in the PR ?    If yes,  it will make
> KIP-236
> > > (it
> > > > > > > has PR already) trivial, since the cancel all pending
> > > reassignments,
> > > > > > > one just needs to do a ListPartitionRessignmentRequest, then
> submit
> > > > > > > empty replicas for all those topic/partitions in
> > > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > George
> > > > > > >
> > > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > > > <cm...@apache.org> wrote:
> > > > > > >
> > > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > > > Hi Colin,
> > > > > > > > >
> > > > > > > > > I think storing reassignment state at the partition level
> is
> > > the
> > > > > right move
> > > > > > > > > and I also agree that replicas should understand that
> there is
> > > a
> > > > > > > > > reassignment in progress. This makes KIP-352 a trivial
> > > follow-up
> > > > > for
> > > > > > > > > example. The only doubt I have is whether the leader and
> isr
> > > znode
> > > > > is the
> > > > > > > > > right place to store the target reassignment. It is a bit
> odd
> > > to
> > > > > keep the
> > > > > > > > > target assignment in a separate place from the current
> > > assignment,
> > > > > right? I
> > > > > > > > > assume the thinking is probably that although the current
> > > > > assignment should
> > > > > > > > > probably be in the leader and isr znode as well, it is
> hard to
> > > > > move the
> > > > > > > > > state in a compatible way. Is that right? But if we have no
> > > plan
> > > > > to remove
> > > > > > > > > the assignment znode, do you see a downside to storing the
> > > target
> > > > > > > > > assignment there as well?
> > > > > > > > >
> > > > > > > >
> > > > > > > > Hi Jason,
> > > > > > > >
> > > > > > > > That's a good point -- it's probably better to keep the
> target
> > > > > > > > assignment in the same znode as the current assignment, for
> > > > > > > > consistency.  I'll change the KIP.
> > > > > > >
> > > > > > > Hi Jason,
> > > > > > >
> > > > > > > Thanks again for the review.
> > > > > > >
> > > > > > > I took another look at this, and I think we should stick with
> the
> > > > > > > initial proposal of putting the reassignment state into
> > > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The
> > > reason is
> > > > > > > because we'll want to bump the leader epoch for the partition
> when
> > > > > > > changing the reassignment state, and the leader epoch resides
> in
> > > that
> > > > > > > znode anyway.  I agree there is some inconsistency here, but
> so be
> > > it:
> > > > > > > if we were to greenfield these zookeeper data structures, we
> might
> > > do
> > > > > > > it differently, but the proposed scheme will work fine and be
> > > > > > > extensible for the future.
> > > > > > >
> > > > > > > >
> > > > > > > > > A few additional questions:
> > > > > > > > >
> > > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > > `alterPartitionAssignments`?
> > > > > > > > > It's the current assignment we're altering, right?
> > > > > > > >
> > > > > > > > That's fair.  AlterPartitionAssigments reads a little
> better, and
> > > > > I'll
> > > > > > > > change it to that.
> > > > > > >
> > > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > > >
> > > > > > > >
> > > > > > > > > 2. Does this change affect the Metadata API? In other
> words,
> > > are
> > > > > clients
> > > > > > > > > aware of reassignments? If so, then we probably need a
> change
> > > to
> > > > > > > > > UpdateMetadata as well. The only alternative I can think of
> > > would
> > > > > be to
> > > > > > > > > represent the replica set in the Metadata request as the
> union
> > > of
> > > > > the
> > > > > > > > > current and target replicas, but I can't think of any
> benefit
> > > to
> > > > > hiding
> > > > > > > > > reassignments. Note that if we did this, we probably
> wouldn't
> > > need
> > > > > a
> > > > > > > > > separate API to list reassignments.
> > > > > > > >
> > > > > > > > I thought about this a bit... and I think on balance, you're
> > > right.
> > > > > We
> > > > > > > > should keep this information together with the replica
> nodes, isr
> > > > > > > > nodes, and offline replicas, and that information is
> available in
> > > > > the
> > > > > > > > MetadataResponse.
> > > > > > > >  However, I do think in order to do this, we'll need a flag
> in
> > > the
> > > > > > > > MetadataRequest that specifiies "only show me reassigning
> > > > > partitions".
> > > > > > > > I'll add this.
> > > > > > >
> > > > > > > I revisited this, and I think we should stick with the original
> > > > > > > proposal of having a separate ListPartitionReassignments API.
> > > There
> > > > > > > really is no use case where the Producer or Consumer needs to
> know
> > > > > > > about a reassignment.  They should just be notified when the
> set of
> > > > > > > partitions changes, which doesn't require changes to
> > > > > > > MetadataRequest/Response.  The Admin client only cares if
> someone
> > > is
> > > > > > > managing the reassignment.  So adding this state to the
> > > > > > > MetadataResponse adds overhead for no real benefit.  In the
> common
> > > > > case
> > > > > > > where there is no ongoing reassignment, it would be 4 bytes per
> > > > > > > partition of extra overhead in the MetadataResponse.
> > > > > > >
> > > > > > > In general, I think we have a problem of oversharing in the
> > > > > > > MetadataRequest/Response.  As we 10x or 100x the number of
> > > partitions
> > > > > > > we support, we'll need to get stricter about giving clients
> only
> > > the
> > > > > > > information they actually need, about the partitions they
> actually
> > > > > care
> > > > > > > about.  Reassignment state clearly falls in the category of
> state
> > > that
> > > > > > > isn't needed by clients (except very specialized rebalancing
> > > programs).
> > > > > > >
> > > > > > > Another important consideration here is that someone managing
> an
> > > > > > > ongoing reassignment wants the most up-to-date information,
> which
> > > is
> > > > > to
> > > > > > > be found on the controller.  Therefore adding this state to
> > > listTopics
> > > > > > > or describeTopics, which could contact any node in the
> cluster, is
> > > > > > > sub-optimal.
> > > > > > >
> > > > > > > Finally, adding this to listTopics or describeTopics feels
> like a
> > > > > warty
> > > > > > > API.  It's an extra boolean which interacts with other extra
> > > booleans
> > > > > > > like "show internal", etc. in weird ways.  I think a separate
> API
> > > is
> > > > > > > cleaner.
> > > > > > >
> > > > > > > >
> > > > > > > > > 3. As replicas come into sync, they will join the ISR.
> Will we
> > > > > await all
> > > > > > > > > target replicas joining the ISR before taking the replica
> out
> > > of
> > > > > the target
> > > > > > > > > replicas set? Also, I assume that target replicas can
> still be
> > > > > elected as
> > > > > > > > > leader?
> > > > > > > >
> > > > > > > > We'll take a replica out of the target replicas set as soon
> as
> > > that
> > > > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > > > >
> > > > > > > > > 4. Probably useful to mention permissions for the new APIs.
> > > > > > > >
> > > > > > > > Good point.  I think alterPartitionAssignments should require
> > > ALTER
> > > > > on
> > > > > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > > > > >
> > > > > > > I added permission information.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Jason
> > > > > > > > >
> > > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > > gwen@confluent.io>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 (binding)
> > > > > > > > > > Looks great, and will be awesome to have this new
> capability.
> > > > > > > > > >
> > > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > > cmccabe@apache.org>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi all,
> > > > > > > > > > >
> > > > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > > > Administrative API for
> > > > > > > > > > > Replica Reassignment.  I think this KIP is important
> since
> > > it
> > > > > will unlock
> > > > > > > > > > > many follow-on improvements to Kafka reassignment (see
> the
> > > > > "Future work"
> > > > > > > > > > > section, plus a lot of the other discussions we've had
> > > > > recently about
> > > > > > > > > > > reassignment).  It also furthers the important KIP-4
> goal
> > > of
> > > > > removing
> > > > > > > > > > > direct access to ZK.
> > > > > > > > > > >
> > > > > > > > > > > I made a few changes based on the discussion in the
> > > [DISCUSS]
> > > > > thread.  As
> > > > > > > > > > > Robert suggested, I removed the need to explicitly
> cancel a
> > > > > reassignment
> > > > > > > > > > > for a partition before setting up a different
> reassignment
> > > for
> > > > > that
> > > > > > > > > > > specific partition.  I also simplified the API a bit by
> > > adding
> > > > > a
> > > > > > > > > > > PartitionReassignment class which is used by both the
> alter
> > > > > and list
> > > > > > > > > > APIs.
> > > > > > > > > > >
> > > > > > > > > > > I modified the proposal so that we now deprecate the
> old
> > > > > znode-based API
> > > > > > > > > > > rather than removing it completely.  That should give
> > > external
> > > > > > > > > > rebalancing
> > > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > > >
> > > > > > > > > > > To clarify a question Viktor asked, I added a note
> that the
> > > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > > --bootstrap-server
> > > > > argument
> > > > > > > > > > to
> > > > > > > > > > > contact the admin APIs.
> > > > > > > > > > >
> > > > > > > > > > > thanks,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > --
> > > > > > > > > > *Gwen Shapira*
> > > > > > > > > > Product Manager | Confluent
> > > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> blog
> > > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
>
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by George Li <sq...@yahoo.com.INVALID>.
 +1 (non-binding)



Thanks for addressing the comments. 
George

    On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <gw...@confluent.io> wrote:  
 
 Renewing my +1, thank you Colin and Stan for working through all the
questions, edge cases, requests and alternatives. We ended up with a
great protocol.

On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <ja...@confluent.io> wrote:
>
> +1 Thanks for the KIP. Really looking forward to this!
>
> -Jason
>
> On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org> wrote:
>
> > Thanks, Stanislav.  Let's restart the vote to reflect the fact that we've
> > made significant changes.  The new vote will go for 3 days as usual.
> >
> > I'll start with my +1 (binding).
> >
> > best,
> > Colin
> >
> >
> > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > Hey everybody,
> > >
> > > We have further iterated on the KIP in the accompanying discussion thread
> > > and I'd like to propose we resume the vote.
> > >
> > > Some notable changes:
> > > - we will store reassignment information in the `/brokers/topics/[topic]`
> > > - we will internally use two collections to represent a reassignment -
> > > "addingReplicas" and "removingReplicas". LeaderAndIsr has been updated
> > > accordingly
> > > - the Alter API will still use the "targetReplicas" collection, but the
> > > List API will now return three separate collections - the full replica
> > set,
> > > the replicas we are adding as part of this reassignment
> > ("addingReplicas")
> > > and the replicas we are removing ("removingReplicas")
> > > - cancellation of a reassignment now means a proper rollback of the
> > > assignment to its original state prior to the API call
> > >
> > > As always, you can re-read the KIP here
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > >
> > > Best,
> > > Stanislav
> > >
> > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi George,
> > > >
> > > > Thanks for taking a look.  I am working on getting a PR done as a
> > > > proof-of-concept.  I'll post it soon.  Then we'll finish up the vote.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > >  Hi Colin,
> > > > >
> > > > >  Great! Looking forward to these features.    +1 (non-binding)
> > > > >
> > > > > What is the estimated timeline to have this implemented?  If any help
> > > > > is needed in the implementation of cancelling reassignments,  I can
> > > > > help if there is spare cycle.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > George
> > > > >
> > > > >
> > > > >
> > > > >    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > > <cm...@apache.org> wrote:
> > > > >
> > > > >  Hi George,
> > > > >
> > > > > Yes, KIP-455 allows the reassignment of individual partitions to be
> > > > > cancelled.  I think it's very important for these operations to be at
> > > > > the partition level.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > >  Hi Colin,
> > > > > >
> > > > > > Thanks for the updated KIP.  It has very good improvements of Kafka
> > > > > > reassignment operations.
> > > > > >
> > > > > > One question, looks like the KIP includes the Cancellation of
> > > > > > individual pending reassignments as well when the
> > > > > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > > > > topic/partition. Will you also be implementing the the partition
> > > > > > cancellation/rollback in the PR ?    If yes,  it will make KIP-236
> > (it
> > > > > > has PR already) trivial, since the cancel all pending
> > reassignments,
> > > > > > one just needs to do a ListPartitionRessignmentRequest, then submit
> > > > > > empty replicas for all those topic/partitions in
> > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > George
> > > > > >
> > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > > <cm...@apache.org> wrote:
> > > > > >
> > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > > Hi Colin,
> > > > > > > >
> > > > > > > > I think storing reassignment state at the partition level is
> > the
> > > > right move
> > > > > > > > and I also agree that replicas should understand that there is
> > a
> > > > > > > > reassignment in progress. This makes KIP-352 a trivial
> > follow-up
> > > > for
> > > > > > > > example. The only doubt I have is whether the leader and isr
> > znode
> > > > is the
> > > > > > > > right place to store the target reassignment. It is a bit odd
> > to
> > > > keep the
> > > > > > > > target assignment in a separate place from the current
> > assignment,
> > > > right? I
> > > > > > > > assume the thinking is probably that although the current
> > > > assignment should
> > > > > > > > probably be in the leader and isr znode as well, it is hard to
> > > > move the
> > > > > > > > state in a compatible way. Is that right? But if we have no
> > plan
> > > > to remove
> > > > > > > > the assignment znode, do you see a downside to storing the
> > target
> > > > > > > > assignment there as well?
> > > > > > > >
> > > > > > >
> > > > > > > Hi Jason,
> > > > > > >
> > > > > > > That's a good point -- it's probably better to keep the target
> > > > > > > assignment in the same znode as the current assignment, for
> > > > > > > consistency.  I'll change the KIP.
> > > > > >
> > > > > > Hi Jason,
> > > > > >
> > > > > > Thanks again for the review.
> > > > > >
> > > > > > I took another look at this, and I think we should stick with the
> > > > > > initial proposal of putting the reassignment state into
> > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The
> > reason is
> > > > > > because we'll want to bump the leader epoch for the partition when
> > > > > > changing the reassignment state, and the leader epoch resides in
> > that
> > > > > > znode anyway.  I agree there is some inconsistency here, but so be
> > it:
> > > > > > if we were to greenfield these zookeeper data structures, we might
> > do
> > > > > > it differently, but the proposed scheme will work fine and be
> > > > > > extensible for the future.
> > > > > >
> > > > > > >
> > > > > > > > A few additional questions:
> > > > > > > >
> > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > `alterPartitionAssignments`?
> > > > > > > > It's the current assignment we're altering, right?
> > > > > > >
> > > > > > > That's fair.  AlterPartitionAssigments reads a little better, and
> > > > I'll
> > > > > > > change it to that.
> > > > > >
> > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > >
> > > > > > >
> > > > > > > > 2. Does this change affect the Metadata API? In other words,
> > are
> > > > clients
> > > > > > > > aware of reassignments? If so, then we probably need a change
> > to
> > > > > > > > UpdateMetadata as well. The only alternative I can think of
> > would
> > > > be to
> > > > > > > > represent the replica set in the Metadata request as the union
> > of
> > > > the
> > > > > > > > current and target replicas, but I can't think of any benefit
> > to
> > > > hiding
> > > > > > > > reassignments. Note that if we did this, we probably wouldn't
> > need
> > > > a
> > > > > > > > separate API to list reassignments.
> > > > > > >
> > > > > > > I thought about this a bit... and I think on balance, you're
> > right.
> > > > We
> > > > > > > should keep this information together with the replica nodes, isr
> > > > > > > nodes, and offline replicas, and that information is available in
> > > > the
> > > > > > > MetadataResponse.
> > > > > > >  However, I do think in order to do this, we'll need a flag in
> > the
> > > > > > > MetadataRequest that specifiies "only show me reassigning
> > > > partitions".
> > > > > > > I'll add this.
> > > > > >
> > > > > > I revisited this, and I think we should stick with the original
> > > > > > proposal of having a separate ListPartitionReassignments API.
> > There
> > > > > > really is no use case where the Producer or Consumer needs to know
> > > > > > about a reassignment.  They should just be notified when the set of
> > > > > > partitions changes, which doesn't require changes to
> > > > > > MetadataRequest/Response.  The Admin client only cares if someone
> > is
> > > > > > managing the reassignment.  So adding this state to the
> > > > > > MetadataResponse adds overhead for no real benefit.  In the common
> > > > case
> > > > > > where there is no ongoing reassignment, it would be 4 bytes per
> > > > > > partition of extra overhead in the MetadataResponse.
> > > > > >
> > > > > > In general, I think we have a problem of oversharing in the
> > > > > > MetadataRequest/Response.  As we 10x or 100x the number of
> > partitions
> > > > > > we support, we'll need to get stricter about giving clients only
> > the
> > > > > > information they actually need, about the partitions they actually
> > > > care
> > > > > > about.  Reassignment state clearly falls in the category of state
> > that
> > > > > > isn't needed by clients (except very specialized rebalancing
> > programs).
> > > > > >
> > > > > > Another important consideration here is that someone managing an
> > > > > > ongoing reassignment wants the most up-to-date information, which
> > is
> > > > to
> > > > > > be found on the controller.  Therefore adding this state to
> > listTopics
> > > > > > or describeTopics, which could contact any node in the cluster, is
> > > > > > sub-optimal.
> > > > > >
> > > > > > Finally, adding this to listTopics or describeTopics feels like a
> > > > warty
> > > > > > API.  It's an extra boolean which interacts with other extra
> > booleans
> > > > > > like "show internal", etc. in weird ways.  I think a separate API
> > is
> > > > > > cleaner.
> > > > > >
> > > > > > >
> > > > > > > > 3. As replicas come into sync, they will join the ISR. Will we
> > > > await all
> > > > > > > > target replicas joining the ISR before taking the replica out
> > of
> > > > the target
> > > > > > > > replicas set? Also, I assume that target replicas can still be
> > > > elected as
> > > > > > > > leader?
> > > > > > >
> > > > > > > We'll take a replica out of the target replicas set as soon as
> > that
> > > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > > >
> > > > > > > > 4. Probably useful to mention permissions for the new APIs.
> > > > > > >
> > > > > > > Good point.  I think alterPartitionAssignments should require
> > ALTER
> > > > on
> > > > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > > > >
> > > > > > I added permission information.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Jason
> > > > > > > >
> > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > gwen@confluent.io>
> > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (binding)
> > > > > > > > > Looks great, and will be awesome to have this new capability.
> > > > > > > > >
> > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > cmccabe@apache.org>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > > Administrative API for
> > > > > > > > > > Replica Reassignment.  I think this KIP is important since
> > it
> > > > will unlock
> > > > > > > > > > many follow-on improvements to Kafka reassignment (see the
> > > > "Future work"
> > > > > > > > > > section, plus a lot of the other discussions we've had
> > > > recently about
> > > > > > > > > > reassignment).  It also furthers the important KIP-4 goal
> > of
> > > > removing
> > > > > > > > > > direct access to ZK.
> > > > > > > > > >
> > > > > > > > > > I made a few changes based on the discussion in the
> > [DISCUSS]
> > > > thread.  As
> > > > > > > > > > Robert suggested, I removed the need to explicitly cancel a
> > > > reassignment
> > > > > > > > > > for a partition before setting up a different reassignment
> > for
> > > > that
> > > > > > > > > > specific partition.  I also simplified the API a bit by
> > adding
> > > > a
> > > > > > > > > > PartitionReassignment class which is used by both the alter
> > > > and list
> > > > > > > > > APIs.
> > > > > > > > > >
> > > > > > > > > > I modified the proposal so that we now deprecate the old
> > > > znode-based API
> > > > > > > > > > rather than removing it completely.  That should give
> > external
> > > > > > > > > rebalancing
> > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > >
> > > > > > > > > > To clarify a question Viktor asked, I added a note that the
> > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > --bootstrap-server
> > > > argument
> > > > > > > > > to
> > > > > > > > > > contact the admin APIs.
> > > > > > > > > >
> > > > > > > > > > thanks,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > *Gwen Shapira*
> > > > > > > > > Product Manager | Confluent
> > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >



-- 
Gwen Shapira
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog
  

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Gwen Shapira <gw...@confluent.io>.
Renewing my +1, thank you Colin and Stan for working through all the
questions, edge cases, requests and alternatives. We ended up with a
great protocol.

On Thu, Jul 18, 2019 at 4:54 PM Jason Gustafson <ja...@confluent.io> wrote:
>
> +1 Thanks for the KIP. Really looking forward to this!
>
> -Jason
>
> On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org> wrote:
>
> > Thanks, Stanislav.  Let's restart the vote to reflect the fact that we've
> > made significant changes.  The new vote will go for 3 days as usual.
> >
> > I'll start with my +1 (binding).
> >
> > best,
> > Colin
> >
> >
> > On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > > Hey everybody,
> > >
> > > We have further iterated on the KIP in the accompanying discussion thread
> > > and I'd like to propose we resume the vote.
> > >
> > > Some notable changes:
> > > - we will store reassignment information in the `/brokers/topics/[topic]`
> > > - we will internally use two collections to represent a reassignment -
> > > "addingReplicas" and "removingReplicas". LeaderAndIsr has been updated
> > > accordingly
> > > - the Alter API will still use the "targetReplicas" collection, but the
> > > List API will now return three separate collections - the full replica
> > set,
> > > the replicas we are adding as part of this reassignment
> > ("addingReplicas")
> > > and the replicas we are removing ("removingReplicas")
> > > - cancellation of a reassignment now means a proper rollback of the
> > > assignment to its original state prior to the API call
> > >
> > > As always, you can re-read the KIP here
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > >
> > > Best,
> > > Stanislav
> > >
> > > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi George,
> > > >
> > > > Thanks for taking a look.  I am working on getting a PR done as a
> > > > proof-of-concept.  I'll post it soon.  Then we'll finish up the vote.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > > >  Hi Colin,
> > > > >
> > > > >  Great! Looking forward to these features.    +1 (non-binding)
> > > > >
> > > > > What is the estimated timeline to have this implemented?  If any help
> > > > > is needed in the implementation of cancelling reassignments,  I can
> > > > > help if there is spare cycle.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > George
> > > > >
> > > > >
> > > > >
> > > > >     On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > > <cm...@apache.org> wrote:
> > > > >
> > > > >  Hi George,
> > > > >
> > > > > Yes, KIP-455 allows the reassignment of individual partitions to be
> > > > > cancelled.  I think it's very important for these operations to be at
> > > > > the partition level.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > > >  Hi Colin,
> > > > > >
> > > > > > Thanks for the updated KIP.  It has very good improvements of Kafka
> > > > > > reassignment operations.
> > > > > >
> > > > > > One question, looks like the KIP includes the Cancellation of
> > > > > > individual pending reassignments as well when the
> > > > > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > > > > topic/partition. Will you also be implementing the the partition
> > > > > > cancellation/rollback in the PR ?    If yes,  it will make KIP-236
> > (it
> > > > > > has PR already) trivial, since the cancel all pending
> > reassignments,
> > > > > > one just needs to do a ListPartitionRessignmentRequest, then submit
> > > > > > empty replicas for all those topic/partitions in
> > > > > > one AlterPartitionReasisgnmentRequest.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > George
> > > > > >
> > > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > > <cm...@apache.org> wrote:
> > > > > >
> > > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > > Hi Colin,
> > > > > > > >
> > > > > > > > I think storing reassignment state at the partition level is
> > the
> > > > right move
> > > > > > > > and I also agree that replicas should understand that there is
> > a
> > > > > > > > reassignment in progress. This makes KIP-352 a trivial
> > follow-up
> > > > for
> > > > > > > > example. The only doubt I have is whether the leader and isr
> > znode
> > > > is the
> > > > > > > > right place to store the target reassignment. It is a bit odd
> > to
> > > > keep the
> > > > > > > > target assignment in a separate place from the current
> > assignment,
> > > > right? I
> > > > > > > > assume the thinking is probably that although the current
> > > > assignment should
> > > > > > > > probably be in the leader and isr znode as well, it is hard to
> > > > move the
> > > > > > > > state in a compatible way. Is that right? But if we have no
> > plan
> > > > to remove
> > > > > > > > the assignment znode, do you see a downside to storing the
> > target
> > > > > > > > assignment there as well?
> > > > > > > >
> > > > > > >
> > > > > > > Hi Jason,
> > > > > > >
> > > > > > > That's a good point -- it's probably better to keep the target
> > > > > > > assignment in the same znode as the current assignment, for
> > > > > > > consistency.  I'll change the KIP.
> > > > > >
> > > > > > Hi Jason,
> > > > > >
> > > > > > Thanks again for the review.
> > > > > >
> > > > > > I took another look at this, and I think we should stick with the
> > > > > > initial proposal of putting the reassignment state into
> > > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The
> > reason is
> > > > > > because we'll want to bump the leader epoch for the partition when
> > > > > > changing the reassignment state, and the leader epoch resides in
> > that
> > > > > > znode anyway.  I agree there is some inconsistency here, but so be
> > it:
> > > > > > if we were to greenfield these zookeeper data structures, we might
> > do
> > > > > > it differently, but the proposed scheme will work fine and be
> > > > > > extensible for the future.
> > > > > >
> > > > > > >
> > > > > > > > A few additional questions:
> > > > > > > >
> > > > > > > > 1. Should `alterPartitionReassignments` be
> > > > `alterPartitionAssignments`?
> > > > > > > > It's the current assignment we're altering, right?
> > > > > > >
> > > > > > > That's fair.  AlterPartitionAssigments reads a little better, and
> > > > I'll
> > > > > > > change it to that.
> > > > > >
> > > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > > >
> > > > > > >
> > > > > > > > 2. Does this change affect the Metadata API? In other words,
> > are
> > > > clients
> > > > > > > > aware of reassignments? If so, then we probably need a change
> > to
> > > > > > > > UpdateMetadata as well. The only alternative I can think of
> > would
> > > > be to
> > > > > > > > represent the replica set in the Metadata request as the union
> > of
> > > > the
> > > > > > > > current and target replicas, but I can't think of any benefit
> > to
> > > > hiding
> > > > > > > > reassignments. Note that if we did this, we probably wouldn't
> > need
> > > > a
> > > > > > > > separate API to list reassignments.
> > > > > > >
> > > > > > > I thought about this a bit... and I think on balance, you're
> > right.
> > > > We
> > > > > > > should keep this information together with the replica nodes, isr
> > > > > > > nodes, and offline replicas, and that information is available in
> > > > the
> > > > > > > MetadataResponse.
> > > > > > >  However, I do think in order to do this, we'll need a flag in
> > the
> > > > > > > MetadataRequest that specifiies "only show me reassigning
> > > > partitions".
> > > > > > > I'll add this.
> > > > > >
> > > > > > I revisited this, and I think we should stick with the original
> > > > > > proposal of having a separate ListPartitionReassignments API.
> > There
> > > > > > really is no use case where the Producer or Consumer needs to know
> > > > > > about a reassignment.  They should just be notified when the set of
> > > > > > partitions changes, which doesn't require changes to
> > > > > > MetadataRequest/Response.  The Admin client only cares if someone
> > is
> > > > > > managing the reassignment.  So adding this state to the
> > > > > > MetadataResponse adds overhead for no real benefit.  In the common
> > > > case
> > > > > > where there is no ongoing reassignment, it would be 4 bytes per
> > > > > > partition of extra overhead in the MetadataResponse.
> > > > > >
> > > > > > In general, I think we have a problem of oversharing in the
> > > > > > MetadataRequest/Response.  As we 10x or 100x the number of
> > partitions
> > > > > > we support, we'll need to get stricter about giving clients only
> > the
> > > > > > information they actually need, about the partitions they actually
> > > > care
> > > > > > about.  Reassignment state clearly falls in the category of state
> > that
> > > > > > isn't needed by clients (except very specialized rebalancing
> > programs).
> > > > > >
> > > > > > Another important consideration here is that someone managing an
> > > > > > ongoing reassignment wants the most up-to-date information, which
> > is
> > > > to
> > > > > > be found on the controller.  Therefore adding this state to
> > listTopics
> > > > > > or describeTopics, which could contact any node in the cluster, is
> > > > > > sub-optimal.
> > > > > >
> > > > > > Finally, adding this to listTopics or describeTopics feels like a
> > > > warty
> > > > > > API.  It's an extra boolean which interacts with other extra
> > booleans
> > > > > > like "show internal", etc. in weird ways.  I think a separate API
> > is
> > > > > > cleaner.
> > > > > >
> > > > > > >
> > > > > > > > 3. As replicas come into sync, they will join the ISR. Will we
> > > > await all
> > > > > > > > target replicas joining the ISR before taking the replica out
> > of
> > > > the target
> > > > > > > > replicas set? Also, I assume that target replicas can still be
> > > > elected as
> > > > > > > > leader?
> > > > > > >
> > > > > > > We'll take a replica out of the target replicas set as soon as
> > that
> > > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > > >
> > > > > > > > 4. Probably useful to mention permissions for the new APIs.
> > > > > > >
> > > > > > > Good point.  I think alterPartitionAssignments should require
> > ALTER
> > > > on
> > > > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > > > >
> > > > > > I added permission information.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Jason
> > > > > > > >
> > > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> > gwen@confluent.io>
> > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (binding)
> > > > > > > > > Looks great, and will be awesome to have this new capability.
> > > > > > > > >
> > > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> > cmccabe@apache.org>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > > Administrative API for
> > > > > > > > > > Replica Reassignment.  I think this KIP is important since
> > it
> > > > will unlock
> > > > > > > > > > many follow-on improvements to Kafka reassignment (see the
> > > > "Future work"
> > > > > > > > > > section, plus a lot of the other discussions we've had
> > > > recently about
> > > > > > > > > > reassignment).  It also furthers the important KIP-4 goal
> > of
> > > > removing
> > > > > > > > > > direct access to ZK.
> > > > > > > > > >
> > > > > > > > > > I made a few changes based on the discussion in the
> > [DISCUSS]
> > > > thread.  As
> > > > > > > > > > Robert suggested, I removed the need to explicitly cancel a
> > > > reassignment
> > > > > > > > > > for a partition before setting up a different reassignment
> > for
> > > > that
> > > > > > > > > > specific partition.  I also simplified the API a bit by
> > adding
> > > > a
> > > > > > > > > > PartitionReassignment class which is used by both the alter
> > > > and list
> > > > > > > > > APIs.
> > > > > > > > > >
> > > > > > > > > > I modified the proposal so that we now deprecate the old
> > > > znode-based API
> > > > > > > > > > rather than removing it completely.  That should give
> > external
> > > > > > > > > rebalancing
> > > > > > > > > > tools some time to transition to the new API.
> > > > > > > > > >
> > > > > > > > > > To clarify a question Viktor asked, I added a note that the
> > > > > > > > > > kafka-reassign-partitions.sh will now use a
> > --bootstrap-server
> > > > argument
> > > > > > > > > to
> > > > > > > > > > contact the admin APIs.
> > > > > > > > > >
> > > > > > > > > > thanks,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > *Gwen Shapira*
> > > > > > > > > Product Manager | Confluent
> > > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >



-- 
Gwen Shapira
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Jason Gustafson <ja...@confluent.io>.
+1 Thanks for the KIP. Really looking forward to this!

-Jason

On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe <cm...@apache.org> wrote:

> Thanks, Stanislav.  Let's restart the vote to reflect the fact that we've
> made significant changes.  The new vote will go for 3 days as usual.
>
> I'll start with my +1 (binding).
>
> best,
> Colin
>
>
> On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> > Hey everybody,
> >
> > We have further iterated on the KIP in the accompanying discussion thread
> > and I'd like to propose we resume the vote.
> >
> > Some notable changes:
> > - we will store reassignment information in the `/brokers/topics/[topic]`
> > - we will internally use two collections to represent a reassignment -
> > "addingReplicas" and "removingReplicas". LeaderAndIsr has been updated
> > accordingly
> > - the Alter API will still use the "targetReplicas" collection, but the
> > List API will now return three separate collections - the full replica
> set,
> > the replicas we are adding as part of this reassignment
> ("addingReplicas")
> > and the replicas we are removing ("removingReplicas")
> > - cancellation of a reassignment now means a proper rollback of the
> > assignment to its original state prior to the API call
> >
> > As always, you can re-read the KIP here
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> >
> > Best,
> > Stanislav
> >
> > On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi George,
> > >
> > > Thanks for taking a look.  I am working on getting a PR done as a
> > > proof-of-concept.  I'll post it soon.  Then we'll finish up the vote.
> > >
> > > best,
> > > Colin
> > >
> > > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > > >  Hi Colin,
> > > >
> > > >  Great! Looking forward to these features.    +1 (non-binding)
> > > >
> > > > What is the estimated timeline to have this implemented?  If any help
> > > > is needed in the implementation of cancelling reassignments,  I can
> > > > help if there is spare cycle.
> > > >
> > > >
> > > > Thanks,
> > > > George
> > > >
> > > >
> > > >
> > > >     On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > > <cm...@apache.org> wrote:
> > > >
> > > >  Hi George,
> > > >
> > > > Yes, KIP-455 allows the reassignment of individual partitions to be
> > > > cancelled.  I think it's very important for these operations to be at
> > > > the partition level.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > > >  Hi Colin,
> > > > >
> > > > > Thanks for the updated KIP.  It has very good improvements of Kafka
> > > > > reassignment operations.
> > > > >
> > > > > One question, looks like the KIP includes the Cancellation of
> > > > > individual pending reassignments as well when the
> > > > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > > > topic/partition. Will you also be implementing the the partition
> > > > > cancellation/rollback in the PR ?    If yes,  it will make KIP-236
> (it
> > > > > has PR already) trivial, since the cancel all pending
> reassignments,
> > > > > one just needs to do a ListPartitionRessignmentRequest, then submit
> > > > > empty replicas for all those topic/partitions in
> > > > > one AlterPartitionReasisgnmentRequest.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > George
> > > > >
> > > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > > <cm...@apache.org> wrote:
> > > > >
> > > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > > Hi Colin,
> > > > > > >
> > > > > > > I think storing reassignment state at the partition level is
> the
> > > right move
> > > > > > > and I also agree that replicas should understand that there is
> a
> > > > > > > reassignment in progress. This makes KIP-352 a trivial
> follow-up
> > > for
> > > > > > > example. The only doubt I have is whether the leader and isr
> znode
> > > is the
> > > > > > > right place to store the target reassignment. It is a bit odd
> to
> > > keep the
> > > > > > > target assignment in a separate place from the current
> assignment,
> > > right? I
> > > > > > > assume the thinking is probably that although the current
> > > assignment should
> > > > > > > probably be in the leader and isr znode as well, it is hard to
> > > move the
> > > > > > > state in a compatible way. Is that right? But if we have no
> plan
> > > to remove
> > > > > > > the assignment znode, do you see a downside to storing the
> target
> > > > > > > assignment there as well?
> > > > > > >
> > > > > >
> > > > > > Hi Jason,
> > > > > >
> > > > > > That's a good point -- it's probably better to keep the target
> > > > > > assignment in the same znode as the current assignment, for
> > > > > > consistency.  I'll change the KIP.
> > > > >
> > > > > Hi Jason,
> > > > >
> > > > > Thanks again for the review.
> > > > >
> > > > > I took another look at this, and I think we should stick with the
> > > > > initial proposal of putting the reassignment state into
> > > > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The
> reason is
> > > > > because we'll want to bump the leader epoch for the partition when
> > > > > changing the reassignment state, and the leader epoch resides in
> that
> > > > > znode anyway.  I agree there is some inconsistency here, but so be
> it:
> > > > > if we were to greenfield these zookeeper data structures, we might
> do
> > > > > it differently, but the proposed scheme will work fine and be
> > > > > extensible for the future.
> > > > >
> > > > > >
> > > > > > > A few additional questions:
> > > > > > >
> > > > > > > 1. Should `alterPartitionReassignments` be
> > > `alterPartitionAssignments`?
> > > > > > > It's the current assignment we're altering, right?
> > > > > >
> > > > > > That's fair.  AlterPartitionAssigments reads a little better, and
> > > I'll
> > > > > > change it to that.
> > > > >
> > > > > +1.  I've changed the RPC and API name in the wiki.
> > > > >
> > > > > >
> > > > > > > 2. Does this change affect the Metadata API? In other words,
> are
> > > clients
> > > > > > > aware of reassignments? If so, then we probably need a change
> to
> > > > > > > UpdateMetadata as well. The only alternative I can think of
> would
> > > be to
> > > > > > > represent the replica set in the Metadata request as the union
> of
> > > the
> > > > > > > current and target replicas, but I can't think of any benefit
> to
> > > hiding
> > > > > > > reassignments. Note that if we did this, we probably wouldn't
> need
> > > a
> > > > > > > separate API to list reassignments.
> > > > > >
> > > > > > I thought about this a bit... and I think on balance, you're
> right.
> > > We
> > > > > > should keep this information together with the replica nodes, isr
> > > > > > nodes, and offline replicas, and that information is available in
> > > the
> > > > > > MetadataResponse.
> > > > > >  However, I do think in order to do this, we'll need a flag in
> the
> > > > > > MetadataRequest that specifiies "only show me reassigning
> > > partitions".
> > > > > > I'll add this.
> > > > >
> > > > > I revisited this, and I think we should stick with the original
> > > > > proposal of having a separate ListPartitionReassignments API.
> There
> > > > > really is no use case where the Producer or Consumer needs to know
> > > > > about a reassignment.  They should just be notified when the set of
> > > > > partitions changes, which doesn't require changes to
> > > > > MetadataRequest/Response.  The Admin client only cares if someone
> is
> > > > > managing the reassignment.  So adding this state to the
> > > > > MetadataResponse adds overhead for no real benefit.  In the common
> > > case
> > > > > where there is no ongoing reassignment, it would be 4 bytes per
> > > > > partition of extra overhead in the MetadataResponse.
> > > > >
> > > > > In general, I think we have a problem of oversharing in the
> > > > > MetadataRequest/Response.  As we 10x or 100x the number of
> partitions
> > > > > we support, we'll need to get stricter about giving clients only
> the
> > > > > information they actually need, about the partitions they actually
> > > care
> > > > > about.  Reassignment state clearly falls in the category of state
> that
> > > > > isn't needed by clients (except very specialized rebalancing
> programs).
> > > > >
> > > > > Another important consideration here is that someone managing an
> > > > > ongoing reassignment wants the most up-to-date information, which
> is
> > > to
> > > > > be found on the controller.  Therefore adding this state to
> listTopics
> > > > > or describeTopics, which could contact any node in the cluster, is
> > > > > sub-optimal.
> > > > >
> > > > > Finally, adding this to listTopics or describeTopics feels like a
> > > warty
> > > > > API.  It's an extra boolean which interacts with other extra
> booleans
> > > > > like "show internal", etc. in weird ways.  I think a separate API
> is
> > > > > cleaner.
> > > > >
> > > > > >
> > > > > > > 3. As replicas come into sync, they will join the ISR. Will we
> > > await all
> > > > > > > target replicas joining the ISR before taking the replica out
> of
> > > the target
> > > > > > > replicas set? Also, I assume that target replicas can still be
> > > elected as
> > > > > > > leader?
> > > > > >
> > > > > > We'll take a replica out of the target replicas set as soon as
> that
> > > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > > >
> > > > > > > 4. Probably useful to mention permissions for the new APIs.
> > > > > >
> > > > > > Good point.  I think alterPartitionAssignments should require
> ALTER
> > > on
> > > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > > >
> > > > > I added permission information.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Jason
> > > > > > >
> > > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <
> gwen@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > > > +1 (binding)
> > > > > > > > Looks great, and will be awesome to have this new capability.
> > > > > > > >
> > > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <
> cmccabe@apache.org>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > I'd like to start the vote for KIP-455: Create an
> > > Administrative API for
> > > > > > > > > Replica Reassignment.  I think this KIP is important since
> it
> > > will unlock
> > > > > > > > > many follow-on improvements to Kafka reassignment (see the
> > > "Future work"
> > > > > > > > > section, plus a lot of the other discussions we've had
> > > recently about
> > > > > > > > > reassignment).  It also furthers the important KIP-4 goal
> of
> > > removing
> > > > > > > > > direct access to ZK.
> > > > > > > > >
> > > > > > > > > I made a few changes based on the discussion in the
> [DISCUSS]
> > > thread.  As
> > > > > > > > > Robert suggested, I removed the need to explicitly cancel a
> > > reassignment
> > > > > > > > > for a partition before setting up a different reassignment
> for
> > > that
> > > > > > > > > specific partition.  I also simplified the API a bit by
> adding
> > > a
> > > > > > > > > PartitionReassignment class which is used by both the alter
> > > and list
> > > > > > > > APIs.
> > > > > > > > >
> > > > > > > > > I modified the proposal so that we now deprecate the old
> > > znode-based API
> > > > > > > > > rather than removing it completely.  That should give
> external
> > > > > > > > rebalancing
> > > > > > > > > tools some time to transition to the new API.
> > > > > > > > >
> > > > > > > > > To clarify a question Viktor asked, I added a note that the
> > > > > > > > > kafka-reassign-partitions.sh will now use a
> --bootstrap-server
> > > argument
> > > > > > > > to
> > > > > > > > > contact the admin APIs.
> > > > > > > > >
> > > > > > > > > thanks,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > *Gwen Shapira*
> > > > > > > > Product Manager | Confluent
> > > > > > > > 650.450.2760 | @gwenshap
> > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > > > > <http://www.confluent.io/blog>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Thanks, Stanislav.  Let's restart the vote to reflect the fact that we've made significant changes.  The new vote will go for 3 days as usual.

I'll start with my +1 (binding).

best,
Colin


On Wed, Jul 17, 2019, at 08:56, Stanislav Kozlovski wrote:
> Hey everybody,
> 
> We have further iterated on the KIP in the accompanying discussion thread
> and I'd like to propose we resume the vote.
> 
> Some notable changes:
> - we will store reassignment information in the `/brokers/topics/[topic]`
> - we will internally use two collections to represent a reassignment -
> "addingReplicas" and "removingReplicas". LeaderAndIsr has been updated
> accordingly
> - the Alter API will still use the "targetReplicas" collection, but the
> List API will now return three separate collections - the full replica set,
> the replicas we are adding as part of this reassignment ("addingReplicas")
> and the replicas we are removing ("removingReplicas")
> - cancellation of a reassignment now means a proper rollback of the
> assignment to its original state prior to the API call
> 
> As always, you can re-read the KIP here
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> 
> Best,
> Stanislav
> 
> On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi George,
> >
> > Thanks for taking a look.  I am working on getting a PR done as a
> > proof-of-concept.  I'll post it soon.  Then we'll finish up the vote.
> >
> > best,
> > Colin
> >
> > On Tue, May 21, 2019, at 17:33, George Li wrote:
> > >  Hi Colin,
> > >
> > >  Great! Looking forward to these features.    +1 (non-binding)
> > >
> > > What is the estimated timeline to have this implemented?  If any help
> > > is needed in the implementation of cancelling reassignments,  I can
> > > help if there is spare cycle.
> > >
> > >
> > > Thanks,
> > > George
> > >
> > >
> > >
> > >     On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > > <cm...@apache.org> wrote:
> > >
> > >  Hi George,
> > >
> > > Yes, KIP-455 allows the reassignment of individual partitions to be
> > > cancelled.  I think it's very important for these operations to be at
> > > the partition level.
> > >
> > > best,
> > > Colin
> > >
> > > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > > >  Hi Colin,
> > > >
> > > > Thanks for the updated KIP.  It has very good improvements of Kafka
> > > > reassignment operations.
> > > >
> > > > One question, looks like the KIP includes the Cancellation of
> > > > individual pending reassignments as well when the
> > > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > > topic/partition. Will you also be implementing the the partition
> > > > cancellation/rollback in the PR ?    If yes,  it will make KIP-236 (it
> > > > has PR already) trivial, since the cancel all pending reassignments,
> > > > one just needs to do a ListPartitionRessignmentRequest, then submit
> > > > empty replicas for all those topic/partitions in
> > > > one AlterPartitionReasisgnmentRequest.
> > > >
> > > >
> > > > Thanks,
> > > > George
> > > >
> > > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > > <cm...@apache.org> wrote:
> > > >
> > > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > > Hi Colin,
> > > > > >
> > > > > > I think storing reassignment state at the partition level is the
> > right move
> > > > > > and I also agree that replicas should understand that there is a
> > > > > > reassignment in progress. This makes KIP-352 a trivial follow-up
> > for
> > > > > > example. The only doubt I have is whether the leader and isr znode
> > is the
> > > > > > right place to store the target reassignment. It is a bit odd to
> > keep the
> > > > > > target assignment in a separate place from the current assignment,
> > right? I
> > > > > > assume the thinking is probably that although the current
> > assignment should
> > > > > > probably be in the leader and isr znode as well, it is hard to
> > move the
> > > > > > state in a compatible way. Is that right? But if we have no plan
> > to remove
> > > > > > the assignment znode, do you see a downside to storing the target
> > > > > > assignment there as well?
> > > > > >
> > > > >
> > > > > Hi Jason,
> > > > >
> > > > > That's a good point -- it's probably better to keep the target
> > > > > assignment in the same znode as the current assignment, for
> > > > > consistency.  I'll change the KIP.
> > > >
> > > > Hi Jason,
> > > >
> > > > Thanks again for the review.
> > > >
> > > > I took another look at this, and I think we should stick with the
> > > > initial proposal of putting the reassignment state into
> > > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is
> > > > because we'll want to bump the leader epoch for the partition when
> > > > changing the reassignment state, and the leader epoch resides in that
> > > > znode anyway.  I agree there is some inconsistency here, but so be it:
> > > > if we were to greenfield these zookeeper data structures, we might do
> > > > it differently, but the proposed scheme will work fine and be
> > > > extensible for the future.
> > > >
> > > > >
> > > > > > A few additional questions:
> > > > > >
> > > > > > 1. Should `alterPartitionReassignments` be
> > `alterPartitionAssignments`?
> > > > > > It's the current assignment we're altering, right?
> > > > >
> > > > > That's fair.  AlterPartitionAssigments reads a little better, and
> > I'll
> > > > > change it to that.
> > > >
> > > > +1.  I've changed the RPC and API name in the wiki.
> > > >
> > > > >
> > > > > > 2. Does this change affect the Metadata API? In other words, are
> > clients
> > > > > > aware of reassignments? If so, then we probably need a change to
> > > > > > UpdateMetadata as well. The only alternative I can think of would
> > be to
> > > > > > represent the replica set in the Metadata request as the union of
> > the
> > > > > > current and target replicas, but I can't think of any benefit to
> > hiding
> > > > > > reassignments. Note that if we did this, we probably wouldn't need
> > a
> > > > > > separate API to list reassignments.
> > > > >
> > > > > I thought about this a bit... and I think on balance, you're right.
> > We
> > > > > should keep this information together with the replica nodes, isr
> > > > > nodes, and offline replicas, and that information is available in
> > the
> > > > > MetadataResponse.
> > > > >  However, I do think in order to do this, we'll need a flag in the
> > > > > MetadataRequest that specifiies "only show me reassigning
> > partitions".
> > > > > I'll add this.
> > > >
> > > > I revisited this, and I think we should stick with the original
> > > > proposal of having a separate ListPartitionReassignments API.  There
> > > > really is no use case where the Producer or Consumer needs to know
> > > > about a reassignment.  They should just be notified when the set of
> > > > partitions changes, which doesn't require changes to
> > > > MetadataRequest/Response.  The Admin client only cares if someone is
> > > > managing the reassignment.  So adding this state to the
> > > > MetadataResponse adds overhead for no real benefit.  In the common
> > case
> > > > where there is no ongoing reassignment, it would be 4 bytes per
> > > > partition of extra overhead in the MetadataResponse.
> > > >
> > > > In general, I think we have a problem of oversharing in the
> > > > MetadataRequest/Response.  As we 10x or 100x the number of partitions
> > > > we support, we'll need to get stricter about giving clients only the
> > > > information they actually need, about the partitions they actually
> > care
> > > > about.  Reassignment state clearly falls in the category of state that
> > > > isn't needed by clients (except very specialized rebalancing programs).
> > > >
> > > > Another important consideration here is that someone managing an
> > > > ongoing reassignment wants the most up-to-date information, which is
> > to
> > > > be found on the controller.  Therefore adding this state to listTopics
> > > > or describeTopics, which could contact any node in the cluster, is
> > > > sub-optimal.
> > > >
> > > > Finally, adding this to listTopics or describeTopics feels like a
> > warty
> > > > API.  It's an extra boolean which interacts with other extra booleans
> > > > like "show internal", etc. in weird ways.  I think a separate API is
> > > > cleaner.
> > > >
> > > > >
> > > > > > 3. As replicas come into sync, they will join the ISR. Will we
> > await all
> > > > > > target replicas joining the ISR before taking the replica out of
> > the target
> > > > > > replicas set? Also, I assume that target replicas can still be
> > elected as
> > > > > > leader?
> > > > >
> > > > > We'll take a replica out of the target replicas set as soon as that
> > > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > > >
> > > > > > 4. Probably useful to mention permissions for the new APIs.
> > > > >
> > > > > Good point.  I think alterPartitionAssignments should require ALTER
> > on
> > > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > > >
> > > > I added permission information.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Jason
> > > > > >
> > > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io>
> > wrote:
> > > > > >
> > > > > > > +1 (binding)
> > > > > > > Looks great, and will be awesome to have this new capability.
> > > > > > >
> > > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > I'd like to start the vote for KIP-455: Create an
> > Administrative API for
> > > > > > > > Replica Reassignment.  I think this KIP is important since it
> > will unlock
> > > > > > > > many follow-on improvements to Kafka reassignment (see the
> > "Future work"
> > > > > > > > section, plus a lot of the other discussions we've had
> > recently about
> > > > > > > > reassignment).  It also furthers the important KIP-4 goal of
> > removing
> > > > > > > > direct access to ZK.
> > > > > > > >
> > > > > > > > I made a few changes based on the discussion in the [DISCUSS]
> > thread.  As
> > > > > > > > Robert suggested, I removed the need to explicitly cancel a
> > reassignment
> > > > > > > > for a partition before setting up a different reassignment for
> > that
> > > > > > > > specific partition.  I also simplified the API a bit by adding
> > a
> > > > > > > > PartitionReassignment class which is used by both the alter
> > and list
> > > > > > > APIs.
> > > > > > > >
> > > > > > > > I modified the proposal so that we now deprecate the old
> > znode-based API
> > > > > > > > rather than removing it completely.  That should give external
> > > > > > > rebalancing
> > > > > > > > tools some time to transition to the new API.
> > > > > > > >
> > > > > > > > To clarify a question Viktor asked, I added a note that the
> > > > > > > > kafka-reassign-partitions.sh will now use a --bootstrap-server
> > argument
> > > > > > > to
> > > > > > > > contact the admin APIs.
> > > > > > > >
> > > > > > > > thanks,
> > > > > > > > Colin
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > *Gwen Shapira*
> > > > > > > Product Manager | Confluent
> > > > > > > 650.450.2760 | @gwenshap
> > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > > > <http://www.confluent.io/blog>
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Stanislav Kozlovski <st...@confluent.io>.
Hey everybody,

We have further iterated on the KIP in the accompanying discussion thread
and I'd like to propose we resume the vote.

Some notable changes:
- we will store reassignment information in the `/brokers/topics/[topic]`
- we will internally use two collections to represent a reassignment -
"addingReplicas" and "removingReplicas". LeaderAndIsr has been updated
accordingly
- the Alter API will still use the "targetReplicas" collection, but the
List API will now return three separate collections - the full replica set,
the replicas we are adding as part of this reassignment ("addingReplicas")
and the replicas we are removing ("removingReplicas")
- cancellation of a reassignment now means a proper rollback of the
assignment to its original state prior to the API call

As always, you can re-read the KIP here
https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment

Best,
Stanislav

On Wed, May 22, 2019 at 6:12 PM Colin McCabe <cm...@apache.org> wrote:

> Hi George,
>
> Thanks for taking a look.  I am working on getting a PR done as a
> proof-of-concept.  I'll post it soon.  Then we'll finish up the vote.
>
> best,
> Colin
>
> On Tue, May 21, 2019, at 17:33, George Li wrote:
> >  Hi Colin,
> >
> >  Great! Looking forward to these features.    +1 (non-binding)
> >
> > What is the estimated timeline to have this implemented?  If any help
> > is needed in the implementation of cancelling reassignments,  I can
> > help if there is spare cycle.
> >
> >
> > Thanks,
> > George
> >
> >
> >
> >     On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe
> > <cm...@apache.org> wrote:
> >
> >  Hi George,
> >
> > Yes, KIP-455 allows the reassignment of individual partitions to be
> > cancelled.  I think it's very important for these operations to be at
> > the partition level.
> >
> > best,
> > Colin
> >
> > On Tue, May 14, 2019, at 16:34, George Li wrote:
> > >  Hi Colin,
> > >
> > > Thanks for the updated KIP.  It has very good improvements of Kafka
> > > reassignment operations.
> > >
> > > One question, looks like the KIP includes the Cancellation of
> > > individual pending reassignments as well when the
> > > AlterPartitionReasisgnmentRequest has empty replicas for the
> > > topic/partition. Will you also be implementing the the partition
> > > cancellation/rollback in the PR ?    If yes,  it will make KIP-236 (it
> > > has PR already) trivial, since the cancel all pending reassignments,
> > > one just needs to do a ListPartitionRessignmentRequest, then submit
> > > empty replicas for all those topic/partitions in
> > > one AlterPartitionReasisgnmentRequest.
> > >
> > >
> > > Thanks,
> > > George
> > >
> > >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe
> > > <cm...@apache.org> wrote:
> > >
> > >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > > Hi Colin,
> > > > >
> > > > > I think storing reassignment state at the partition level is the
> right move
> > > > > and I also agree that replicas should understand that there is a
> > > > > reassignment in progress. This makes KIP-352 a trivial follow-up
> for
> > > > > example. The only doubt I have is whether the leader and isr znode
> is the
> > > > > right place to store the target reassignment. It is a bit odd to
> keep the
> > > > > target assignment in a separate place from the current assignment,
> right? I
> > > > > assume the thinking is probably that although the current
> assignment should
> > > > > probably be in the leader and isr znode as well, it is hard to
> move the
> > > > > state in a compatible way. Is that right? But if we have no plan
> to remove
> > > > > the assignment znode, do you see a downside to storing the target
> > > > > assignment there as well?
> > > > >
> > > >
> > > > Hi Jason,
> > > >
> > > > That's a good point -- it's probably better to keep the target
> > > > assignment in the same znode as the current assignment, for
> > > > consistency.  I'll change the KIP.
> > >
> > > Hi Jason,
> > >
> > > Thanks again for the review.
> > >
> > > I took another look at this, and I think we should stick with the
> > > initial proposal of putting the reassignment state into
> > > /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is
> > > because we'll want to bump the leader epoch for the partition when
> > > changing the reassignment state, and the leader epoch resides in that
> > > znode anyway.  I agree there is some inconsistency here, but so be it:
> > > if we were to greenfield these zookeeper data structures, we might do
> > > it differently, but the proposed scheme will work fine and be
> > > extensible for the future.
> > >
> > > >
> > > > > A few additional questions:
> > > > >
> > > > > 1. Should `alterPartitionReassignments` be
> `alterPartitionAssignments`?
> > > > > It's the current assignment we're altering, right?
> > > >
> > > > That's fair.  AlterPartitionAssigments reads a little better, and
> I'll
> > > > change it to that.
> > >
> > > +1.  I've changed the RPC and API name in the wiki.
> > >
> > > >
> > > > > 2. Does this change affect the Metadata API? In other words, are
> clients
> > > > > aware of reassignments? If so, then we probably need a change to
> > > > > UpdateMetadata as well. The only alternative I can think of would
> be to
> > > > > represent the replica set in the Metadata request as the union of
> the
> > > > > current and target replicas, but I can't think of any benefit to
> hiding
> > > > > reassignments. Note that if we did this, we probably wouldn't need
> a
> > > > > separate API to list reassignments.
> > > >
> > > > I thought about this a bit... and I think on balance, you're right.
> We
> > > > should keep this information together with the replica nodes, isr
> > > > nodes, and offline replicas, and that information is available in
> the
> > > > MetadataResponse.
> > > >  However, I do think in order to do this, we'll need a flag in the
> > > > MetadataRequest that specifiies "only show me reassigning
> partitions".
> > > > I'll add this.
> > >
> > > I revisited this, and I think we should stick with the original
> > > proposal of having a separate ListPartitionReassignments API.  There
> > > really is no use case where the Producer or Consumer needs to know
> > > about a reassignment.  They should just be notified when the set of
> > > partitions changes, which doesn't require changes to
> > > MetadataRequest/Response.  The Admin client only cares if someone is
> > > managing the reassignment.  So adding this state to the
> > > MetadataResponse adds overhead for no real benefit.  In the common
> case
> > > where there is no ongoing reassignment, it would be 4 bytes per
> > > partition of extra overhead in the MetadataResponse.
> > >
> > > In general, I think we have a problem of oversharing in the
> > > MetadataRequest/Response.  As we 10x or 100x the number of partitions
> > > we support, we'll need to get stricter about giving clients only the
> > > information they actually need, about the partitions they actually
> care
> > > about.  Reassignment state clearly falls in the category of state that
> > > isn't needed by clients (except very specialized rebalancing programs).
> > >
> > > Another important consideration here is that someone managing an
> > > ongoing reassignment wants the most up-to-date information, which is
> to
> > > be found on the controller.  Therefore adding this state to listTopics
> > > or describeTopics, which could contact any node in the cluster, is
> > > sub-optimal.
> > >
> > > Finally, adding this to listTopics or describeTopics feels like a
> warty
> > > API.  It's an extra boolean which interacts with other extra booleans
> > > like "show internal", etc. in weird ways.  I think a separate API is
> > > cleaner.
> > >
> > > >
> > > > > 3. As replicas come into sync, they will join the ISR. Will we
> await all
> > > > > target replicas joining the ISR before taking the replica out of
> the target
> > > > > replicas set? Also, I assume that target replicas can still be
> elected as
> > > > > leader?
> > > >
> > > > We'll take a replica out of the target replicas set as soon as that
> > > > replica is in the ISR.  Let me clarify this in the KIP.
> > > >
> > > > > 4. Probably useful to mention permissions for the new APIs.
> > > >
> > > > Good point.  I think alterPartitionAssignments should require ALTER
> on
> > > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > >
> > > I added permission information.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io>
> wrote:
> > > > >
> > > > > > +1 (binding)
> > > > > > Looks great, and will be awesome to have this new capability.
> > > > > >
> > > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I'd like to start the vote for KIP-455: Create an
> Administrative API for
> > > > > > > Replica Reassignment.  I think this KIP is important since it
> will unlock
> > > > > > > many follow-on improvements to Kafka reassignment (see the
> "Future work"
> > > > > > > section, plus a lot of the other discussions we've had
> recently about
> > > > > > > reassignment).  It also furthers the important KIP-4 goal of
> removing
> > > > > > > direct access to ZK.
> > > > > > >
> > > > > > > I made a few changes based on the discussion in the [DISCUSS]
> thread.  As
> > > > > > > Robert suggested, I removed the need to explicitly cancel a
> reassignment
> > > > > > > for a partition before setting up a different reassignment for
> that
> > > > > > > specific partition.  I also simplified the API a bit by adding
> a
> > > > > > > PartitionReassignment class which is used by both the alter
> and list
> > > > > > APIs.
> > > > > > >
> > > > > > > I modified the proposal so that we now deprecate the old
> znode-based API
> > > > > > > rather than removing it completely.  That should give external
> > > > > > rebalancing
> > > > > > > tools some time to transition to the new API.
> > > > > > >
> > > > > > > To clarify a question Viktor asked, I added a note that the
> > > > > > > kafka-reassign-partitions.sh will now use a --bootstrap-server
> argument
> > > > > > to
> > > > > > > contact the admin APIs.
> > > > > > >
> > > > > > > thanks,
> > > > > > > Colin
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > *Gwen Shapira*
> > > > > > Product Manager | Confluent
> > > > > > 650.450.2760 | @gwenshap
> > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > > <http://www.confluent.io/blog>
> > > > > >
> > > > >
> > > >
> > >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi George,

Thanks for taking a look.  I am working on getting a PR done as a proof-of-concept.  I'll post it soon.  Then we'll finish up the vote.

best,
Colin

On Tue, May 21, 2019, at 17:33, George Li wrote:
>  Hi Colin,  
> 
>  Great! Looking forward to these features.    +1 (non-binding)
> 
> What is the estimated timeline to have this implemented?  If any help 
> is needed in the implementation of cancelling reassignments,  I can 
> help if there is spare cycle. 
> 
> 
> Thanks,
> George
> 
> 
> 
>     On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe 
> <cm...@apache.org> wrote:  
>  
>  Hi George,
> 
> Yes, KIP-455 allows the reassignment of individual partitions to be 
> cancelled.  I think it's very important for these operations to be at 
> the partition level.
> 
> best,
> Colin
> 
> On Tue, May 14, 2019, at 16:34, George Li wrote:
> >  Hi Colin,
> > 
> > Thanks for the updated KIP.  It has very good improvements of Kafka 
> > reassignment operations. 
> > 
> > One question, looks like the KIP includes the Cancellation of 
> > individual pending reassignments as well when the 
> > AlterPartitionReasisgnmentRequest has empty replicas for the 
> > topic/partition. Will you also be implementing the the partition 
> > cancellation/rollback in the PR ?    If yes,  it will make KIP-236 (it 
> > has PR already) trivial, since the cancel all pending reassignments, 
> > one just needs to do a ListPartitionRessignmentRequest, then submit 
> > empty replicas for all those topic/partitions in 
> > one AlterPartitionReasisgnmentRequest. 
> > 
> > 
> > Thanks,
> > George
> > 
> >    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe 
> > <cm...@apache.org> wrote:  
> >  
> >  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > > Hi Colin,
> > > > 
> > > > I think storing reassignment state at the partition level is the right move
> > > > and I also agree that replicas should understand that there is a
> > > > reassignment in progress. This makes KIP-352 a trivial follow-up for
> > > > example. The only doubt I have is whether the leader and isr znode is the
> > > > right place to store the target reassignment. It is a bit odd to keep the
> > > > target assignment in a separate place from the current assignment, right? I
> > > > assume the thinking is probably that although the current assignment should
> > > > probably be in the leader and isr znode as well, it is hard to move the
> > > > state in a compatible way. Is that right? But if we have no plan to remove
> > > > the assignment znode, do you see a downside to storing the target
> > > > assignment there as well?
> > > >
> > > 
> > > Hi Jason,
> > > 
> > > That's a good point -- it's probably better to keep the target 
> > > assignment in the same znode as the current assignment, for 
> > > consistency.  I'll change the KIP.
> > 
> > Hi Jason,
> > 
> > Thanks again for the review.
> > 
> > I took another look at this, and I think we should stick with the 
> > initial proposal of putting the reassignment state into 
> > /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is 
> > because we'll want to bump the leader epoch for the partition when 
> > changing the reassignment state, and the leader epoch resides in that 
> > znode anyway.  I agree there is some inconsistency here, but so be it: 
> > if we were to greenfield these zookeeper data structures, we might do 
> > it differently, but the proposed scheme will work fine and be 
> > extensible for the future.
> > 
> > > 
> > > > A few additional questions:
> > > > 
> > > > 1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
> > > > It's the current assignment we're altering, right?
> > > 
> > > That's fair.  AlterPartitionAssigments reads a little better, and I'll 
> > > change it to that.
> > 
> > +1.  I've changed the RPC and API name in the wiki.
> > 
> > > 
> > > > 2. Does this change affect the Metadata API? In other words, are clients
> > > > aware of reassignments? If so, then we probably need a change to
> > > > UpdateMetadata as well. The only alternative I can think of would be to
> > > > represent the replica set in the Metadata request as the union of the
> > > > current and target replicas, but I can't think of any benefit to hiding
> > > > reassignments. Note that if we did this, we probably wouldn't need a
> > > > separate API to list reassignments.
> > > 
> > > I thought about this a bit... and I think on balance, you're right.  We 
> > > should keep this information together with the replica nodes, isr 
> > > nodes, and offline replicas, and that information is available in the 
> > > MetadataResponse. 
> > >  However, I do think in order to do this, we'll need a flag in the 
> > > MetadataRequest that specifiies "only show me reassigning partitions".  
> > > I'll add this.
> > 
> > I revisited this, and I think we should stick with the original 
> > proposal of having a separate ListPartitionReassignments API.  There 
> > really is no use case where the Producer or Consumer needs to know 
> > about a reassignment.  They should just be notified when the set of 
> > partitions changes, which doesn't require changes to 
> > MetadataRequest/Response.  The Admin client only cares if someone is 
> > managing the reassignment.  So adding this state to the 
> > MetadataResponse adds overhead for no real benefit.  In the common case 
> > where there is no ongoing reassignment, it would be 4 bytes per 
> > partition of extra overhead in the MetadataResponse.
> > 
> > In general, I think we have a problem of oversharing in the 
> > MetadataRequest/Response.  As we 10x or 100x the number of partitions 
> > we support, we'll need to get stricter about giving clients only the 
> > information they actually need, about the partitions they actually care 
> > about.  Reassignment state clearly falls in the category of state that 
> > isn't needed by clients (except very specialized rebalancing programs).
> > 
> > Another important consideration here is that someone managing an 
> > ongoing reassignment wants the most up-to-date information, which is to 
> > be found on the controller.  Therefore adding this state to listTopics 
> > or describeTopics, which could contact any node in the cluster, is 
> > sub-optimal.
> > 
> > Finally, adding this to listTopics or describeTopics feels like a warty 
> > API.  It's an extra boolean which interacts with other extra booleans 
> > like "show internal", etc. in weird ways.  I think a separate API is 
> > cleaner.
> > 
> > > 
> > > > 3. As replicas come into sync, they will join the ISR. Will we await all
> > > > target replicas joining the ISR before taking the replica out of the target
> > > > replicas set? Also, I assume that target replicas can still be elected as
> > > > leader?
> > > 
> > > We'll take a replica out of the target replicas set as soon as that 
> > > replica is in the ISR.  Let me clarify this in the KIP.
> > > 
> > > > 4. Probably useful to mention permissions for the new APIs.
> > > 
> > > Good point.  I think alterPartitionAssignments should require ALTER on 
> > > CLUSTER.  MetadataRequest permissions will be unchanged.
> > 
> > I added permission information.
> > 
> > best,
> > Colin
> > 
> > > 
> > > best,
> > > Colin
> > > 
> > > > 
> > > > Thanks,
> > > > Jason
> > > > 
> > > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:
> > > > 
> > > > > +1 (binding)
> > > > > Looks great, and will be awesome to have this new capability.
> > > > >
> > > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I'd like to start the vote for KIP-455: Create an Administrative API for
> > > > > > Replica Reassignment.  I think this KIP is important since it will unlock
> > > > > > many follow-on improvements to Kafka reassignment (see the "Future work"
> > > > > > section, plus a lot of the other discussions we've had recently about
> > > > > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > > > > direct access to ZK.
> > > > > >
> > > > > > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > > > > > Robert suggested, I removed the need to explicitly cancel a reassignment
> > > > > > for a partition before setting up a different reassignment for that
> > > > > > specific partition.  I also simplified the API a bit by adding a
> > > > > > PartitionReassignment class which is used by both the alter and list
> > > > > APIs.
> > > > > >
> > > > > > I modified the proposal so that we now deprecate the old znode-based API
> > > > > > rather than removing it completely.  That should give external
> > > > > rebalancing
> > > > > > tools some time to transition to the new API.
> > > > > >
> > > > > > To clarify a question Viktor asked, I added a note that the
> > > > > > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> > > > > to
> > > > > > contact the admin APIs.
> > > > > >
> > > > > > thanks,
> > > > > > Colin
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > *Gwen Shapira*
> > > > > Product Manager | Confluent
> > > > > 650.450.2760 | @gwenshap
> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > <http://www.confluent.io/blog>
> > > > >
> > > >
> > >
> >

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by George Li <sq...@yahoo.com.INVALID>.
 Hi Colin,  

 Great! Looking forward to these features.    +1 (non-binding)

What is the estimated timeline to have this implemented?  If any help is needed in the implementation of cancelling reassignments,  I can help if there is spare cycle. 


Thanks,
George



    On Thursday, May 16, 2019, 9:48:56 AM PDT, Colin McCabe <cm...@apache.org> wrote:  
 
 Hi George,

Yes, KIP-455 allows the reassignment of individual partitions to be cancelled.  I think it's very important for these operations to be at the partition level.

best,
Colin

On Tue, May 14, 2019, at 16:34, George Li wrote:
>  Hi Colin,
> 
> Thanks for the updated KIP.  It has very good improvements of Kafka 
> reassignment operations. 
> 
> One question, looks like the KIP includes the Cancellation of 
> individual pending reassignments as well when the 
> AlterPartitionReasisgnmentRequest has empty replicas for the 
> topic/partition. Will you also be implementing the the partition 
> cancellation/rollback in the PR ?    If yes,  it will make KIP-236 (it 
> has PR already) trivial, since the cancel all pending reassignments, 
> one just needs to do a ListPartitionRessignmentRequest, then submit 
> empty replicas for all those topic/partitions in 
> one AlterPartitionReasisgnmentRequest. 
> 
> 
> Thanks,
> George
> 
>    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe 
> <cm...@apache.org> wrote:  
>  
>  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > Hi Colin,
> > > 
> > > I think storing reassignment state at the partition level is the right move
> > > and I also agree that replicas should understand that there is a
> > > reassignment in progress. This makes KIP-352 a trivial follow-up for
> > > example. The only doubt I have is whether the leader and isr znode is the
> > > right place to store the target reassignment. It is a bit odd to keep the
> > > target assignment in a separate place from the current assignment, right? I
> > > assume the thinking is probably that although the current assignment should
> > > probably be in the leader and isr znode as well, it is hard to move the
> > > state in a compatible way. Is that right? But if we have no plan to remove
> > > the assignment znode, do you see a downside to storing the target
> > > assignment there as well?
> > >
> > 
> > Hi Jason,
> > 
> > That's a good point -- it's probably better to keep the target 
> > assignment in the same znode as the current assignment, for 
> > consistency.  I'll change the KIP.
> 
> Hi Jason,
> 
> Thanks again for the review.
> 
> I took another look at this, and I think we should stick with the 
> initial proposal of putting the reassignment state into 
> /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is 
> because we'll want to bump the leader epoch for the partition when 
> changing the reassignment state, and the leader epoch resides in that 
> znode anyway.  I agree there is some inconsistency here, but so be it: 
> if we were to greenfield these zookeeper data structures, we might do 
> it differently, but the proposed scheme will work fine and be 
> extensible for the future.
> 
> > 
> > > A few additional questions:
> > > 
> > > 1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
> > > It's the current assignment we're altering, right?
> > 
> > That's fair.  AlterPartitionAssigments reads a little better, and I'll 
> > change it to that.
> 
> +1.  I've changed the RPC and API name in the wiki.
> 
> > 
> > > 2. Does this change affect the Metadata API? In other words, are clients
> > > aware of reassignments? If so, then we probably need a change to
> > > UpdateMetadata as well. The only alternative I can think of would be to
> > > represent the replica set in the Metadata request as the union of the
> > > current and target replicas, but I can't think of any benefit to hiding
> > > reassignments. Note that if we did this, we probably wouldn't need a
> > > separate API to list reassignments.
> > 
> > I thought about this a bit... and I think on balance, you're right.  We 
> > should keep this information together with the replica nodes, isr 
> > nodes, and offline replicas, and that information is available in the 
> > MetadataResponse. 
> >  However, I do think in order to do this, we'll need a flag in the 
> > MetadataRequest that specifiies "only show me reassigning partitions".  
> > I'll add this.
> 
> I revisited this, and I think we should stick with the original 
> proposal of having a separate ListPartitionReassignments API.  There 
> really is no use case where the Producer or Consumer needs to know 
> about a reassignment.  They should just be notified when the set of 
> partitions changes, which doesn't require changes to 
> MetadataRequest/Response.  The Admin client only cares if someone is 
> managing the reassignment.  So adding this state to the 
> MetadataResponse adds overhead for no real benefit.  In the common case 
> where there is no ongoing reassignment, it would be 4 bytes per 
> partition of extra overhead in the MetadataResponse.
> 
> In general, I think we have a problem of oversharing in the 
> MetadataRequest/Response.  As we 10x or 100x the number of partitions 
> we support, we'll need to get stricter about giving clients only the 
> information they actually need, about the partitions they actually care 
> about.  Reassignment state clearly falls in the category of state that 
> isn't needed by clients (except very specialized rebalancing programs).
> 
> Another important consideration here is that someone managing an 
> ongoing reassignment wants the most up-to-date information, which is to 
> be found on the controller.  Therefore adding this state to listTopics 
> or describeTopics, which could contact any node in the cluster, is 
> sub-optimal.
> 
> Finally, adding this to listTopics or describeTopics feels like a warty 
> API.  It's an extra boolean which interacts with other extra booleans 
> like "show internal", etc. in weird ways.  I think a separate API is 
> cleaner.
> 
> > 
> > > 3. As replicas come into sync, they will join the ISR. Will we await all
> > > target replicas joining the ISR before taking the replica out of the target
> > > replicas set? Also, I assume that target replicas can still be elected as
> > > leader?
> > 
> > We'll take a replica out of the target replicas set as soon as that 
> > replica is in the ISR.  Let me clarify this in the KIP.
> > 
> > > 4. Probably useful to mention permissions for the new APIs.
> > 
> > Good point.  I think alterPartitionAssignments should require ALTER on 
> > CLUSTER.  MetadataRequest permissions will be unchanged.
> 
> I added permission information.
> 
> best,
> Colin
> 
> > 
> > best,
> > Colin
> > 
> > > 
> > > Thanks,
> > > Jason
> > > 
> > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:
> > > 
> > > > +1 (binding)
> > > > Looks great, and will be awesome to have this new capability.
> > > >
> > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I'd like to start the vote for KIP-455: Create an Administrative API for
> > > > > Replica Reassignment.  I think this KIP is important since it will unlock
> > > > > many follow-on improvements to Kafka reassignment (see the "Future work"
> > > > > section, plus a lot of the other discussions we've had recently about
> > > > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > > > direct access to ZK.
> > > > >
> > > > > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > > > > Robert suggested, I removed the need to explicitly cancel a reassignment
> > > > > for a partition before setting up a different reassignment for that
> > > > > specific partition.  I also simplified the API a bit by adding a
> > > > > PartitionReassignment class which is used by both the alter and list
> > > > APIs.
> > > > >
> > > > > I modified the proposal so that we now deprecate the old znode-based API
> > > > > rather than removing it completely.  That should give external
> > > > rebalancing
> > > > > tools some time to transition to the new API.
> > > > >
> > > > > To clarify a question Viktor asked, I added a note that the
> > > > > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> > > > to
> > > > > contact the admin APIs.
> > > > >
> > > > > thanks,
> > > > > Colin
> > > > >
> > > >
> > > >
> > > > --
> > > > *Gwen Shapira*
> > > > Product Manager | Confluent
> > > > 650.450.2760 | @gwenshap
> > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > <http://www.confluent.io/blog>
> > > >
> > >
> >
>  

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
Hi George,

Yes, KIP-455 allows the reassignment of individual partitions to be cancelled.  I think it's very important for these operations to be at the partition level.

best,
Colin

On Tue, May 14, 2019, at 16:34, George Li wrote:
>  Hi Colin,
> 
> Thanks for the updated KIP.  It has very good improvements of Kafka 
> reassignment operations. 
> 
> One question, looks like the KIP includes the Cancellation of 
> individual pending reassignments as well when the 
> AlterPartitionReasisgnmentRequest has empty replicas for the 
> topic/partition. Will you also be implementing the the partition 
> cancellation/rollback in the PR ?    If yes,  it will make KIP-236 (it 
> has PR already) trivial, since the cancel all pending reassignments, 
> one just needs to do a ListPartitionRessignmentRequest, then submit 
> empty replicas for all those topic/partitions in 
> one AlterPartitionReasisgnmentRequest. 
> 
> 
> Thanks,
> George
> 
>     On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe 
> <cm...@apache.org> wrote:  
>  
>  On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> > On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > > Hi Colin,
> > > 
> > > I think storing reassignment state at the partition level is the right move
> > > and I also agree that replicas should understand that there is a
> > > reassignment in progress. This makes KIP-352 a trivial follow-up for
> > > example. The only doubt I have is whether the leader and isr znode is the
> > > right place to store the target reassignment. It is a bit odd to keep the
> > > target assignment in a separate place from the current assignment, right? I
> > > assume the thinking is probably that although the current assignment should
> > > probably be in the leader and isr znode as well, it is hard to move the
> > > state in a compatible way. Is that right? But if we have no plan to remove
> > > the assignment znode, do you see a downside to storing the target
> > > assignment there as well?
> > >
> > 
> > Hi Jason,
> > 
> > That's a good point -- it's probably better to keep the target 
> > assignment in the same znode as the current assignment, for 
> > consistency.  I'll change the KIP.
> 
> Hi Jason,
> 
> Thanks again for the review.
> 
> I took another look at this, and I think we should stick with the 
> initial proposal of putting the reassignment state into 
> /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is 
> because we'll want to bump the leader epoch for the partition when 
> changing the reassignment state, and the leader epoch resides in that 
> znode anyway.  I agree there is some inconsistency here, but so be it: 
> if we were to greenfield these zookeeper data structures, we might do 
> it differently, but the proposed scheme will work fine and be 
> extensible for the future.
> 
> > 
> > > A few additional questions:
> > > 
> > > 1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
> > > It's the current assignment we're altering, right?
> > 
> > That's fair.  AlterPartitionAssigments reads a little better, and I'll 
> > change it to that.
> 
> +1.  I've changed the RPC and API name in the wiki.
> 
> > 
> > > 2. Does this change affect the Metadata API? In other words, are clients
> > > aware of reassignments? If so, then we probably need a change to
> > > UpdateMetadata as well. The only alternative I can think of would be to
> > > represent the replica set in the Metadata request as the union of the
> > > current and target replicas, but I can't think of any benefit to hiding
> > > reassignments. Note that if we did this, we probably wouldn't need a
> > > separate API to list reassignments.
> > 
> > I thought about this a bit... and I think on balance, you're right.  We 
> > should keep this information together with the replica nodes, isr 
> > nodes, and offline replicas, and that information is available in the 
> > MetadataResponse. 
> >  However, I do think in order to do this, we'll need a flag in the 
> > MetadataRequest that specifiies "only show me reassigning partitions".  
> > I'll add this.
> 
> I revisited this, and I think we should stick with the original 
> proposal of having a separate ListPartitionReassignments API.  There 
> really is no use case where the Producer or Consumer needs to know 
> about a reassignment.  They should just be notified when the set of 
> partitions changes, which doesn't require changes to 
> MetadataRequest/Response.  The Admin client only cares if someone is 
> managing the reassignment.  So adding this state to the 
> MetadataResponse adds overhead for no real benefit.  In the common case 
> where there is no ongoing reassignment, it would be 4 bytes per 
> partition of extra overhead in the MetadataResponse.
> 
> In general, I think we have a problem of oversharing in the 
> MetadataRequest/Response.  As we 10x or 100x the number of partitions 
> we support, we'll need to get stricter about giving clients only the 
> information they actually need, about the partitions they actually care 
> about.  Reassignment state clearly falls in the category of state that 
> isn't needed by clients (except very specialized rebalancing programs).
> 
> Another important consideration here is that someone managing an 
> ongoing reassignment wants the most up-to-date information, which is to 
> be found on the controller.  Therefore adding this state to listTopics 
> or describeTopics, which could contact any node in the cluster, is 
> sub-optimal.
> 
> Finally, adding this to listTopics or describeTopics feels like a warty 
> API.  It's an extra boolean which interacts with other extra booleans 
> like "show internal", etc. in weird ways.  I think a separate API is 
> cleaner.
> 
> > 
> > > 3. As replicas come into sync, they will join the ISR. Will we await all
> > > target replicas joining the ISR before taking the replica out of the target
> > > replicas set? Also, I assume that target replicas can still be elected as
> > > leader?
> > 
> > We'll take a replica out of the target replicas set as soon as that 
> > replica is in the ISR.  Let me clarify this in the KIP.
> > 
> > > 4. Probably useful to mention permissions for the new APIs.
> > 
> > Good point.  I think alterPartitionAssignments should require ALTER on 
> > CLUSTER.  MetadataRequest permissions will be unchanged.
> 
> I added permission information.
> 
> best,
> Colin
> 
> > 
> > best,
> > Colin
> > 
> > > 
> > > Thanks,
> > > Jason
> > > 
> > > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:
> > > 
> > > > +1 (binding)
> > > > Looks great, and will be awesome to have this new capability.
> > > >
> > > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I'd like to start the vote for KIP-455: Create an Administrative API for
> > > > > Replica Reassignment.  I think this KIP is important since it will unlock
> > > > > many follow-on improvements to Kafka reassignment (see the "Future work"
> > > > > section, plus a lot of the other discussions we've had recently about
> > > > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > > > direct access to ZK.
> > > > >
> > > > > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > > > > Robert suggested, I removed the need to explicitly cancel a reassignment
> > > > > for a partition before setting up a different reassignment for that
> > > > > specific partition.  I also simplified the API a bit by adding a
> > > > > PartitionReassignment class which is used by both the alter and list
> > > > APIs.
> > > > >
> > > > > I modified the proposal so that we now deprecate the old znode-based API
> > > > > rather than removing it completely.  That should give external
> > > > rebalancing
> > > > > tools some time to transition to the new API.
> > > > >
> > > > > To clarify a question Viktor asked, I added a note that the
> > > > > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> > > > to
> > > > > contact the admin APIs.
> > > > >
> > > > > thanks,
> > > > > Colin
> > > > >
> > > >
> > > >
> > > > --
> > > > *Gwen Shapira*
> > > > Product Manager | Confluent
> > > > 650.450.2760 | @gwenshap
> > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > <http://www.confluent.io/blog>
> > > >
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by George Li <sq...@yahoo.com.INVALID>.
 Hi Colin,

Thanks for the updated KIP.  It has very good improvements of Kafka reassignment operations. 

One question, looks like the KIP includes the Cancellation of individual pending reassignments as well when the AlterPartitionReasisgnmentRequest has empty replicas for the topic/partition. Will you also be implementing the the partition cancellation/rollback in the PR ?    If yes,  it will make KIP-236 (it has PR already) trivial, since the cancel all pending reassignments, one just needs to do a ListPartitionRessignmentRequest, then submit empty replicas for all those topic/partitions in one AlterPartitionReasisgnmentRequest. 


Thanks,
George

    On Friday, May 10, 2019, 8:44:31 PM PDT, Colin McCabe <cm...@apache.org> wrote:  
 
 On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > Hi Colin,
> > 
> > I think storing reassignment state at the partition level is the right move
> > and I also agree that replicas should understand that there is a
> > reassignment in progress. This makes KIP-352 a trivial follow-up for
> > example. The only doubt I have is whether the leader and isr znode is the
> > right place to store the target reassignment. It is a bit odd to keep the
> > target assignment in a separate place from the current assignment, right? I
> > assume the thinking is probably that although the current assignment should
> > probably be in the leader and isr znode as well, it is hard to move the
> > state in a compatible way. Is that right? But if we have no plan to remove
> > the assignment znode, do you see a downside to storing the target
> > assignment there as well?
> >
> 
> Hi Jason,
> 
> That's a good point -- it's probably better to keep the target 
> assignment in the same znode as the current assignment, for 
> consistency.  I'll change the KIP.

Hi Jason,

Thanks again for the review.

I took another look at this, and I think we should stick with the initial proposal of putting the reassignment state into /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is because we'll want to bump the leader epoch for the partition when changing the reassignment state, and the leader epoch resides in that znode anyway.  I agree there is some inconsistency here, but so be it: if we were to greenfield these zookeeper data structures, we might do it differently, but the proposed scheme will work fine and be extensible for the future.

> 
> > A few additional questions:
> > 
> > 1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
> > It's the current assignment we're altering, right?
> 
> That's fair.  AlterPartitionAssigments reads a little better, and I'll 
> change it to that.

+1.  I've changed the RPC and API name in the wiki.

> 
> > 2. Does this change affect the Metadata API? In other words, are clients
> > aware of reassignments? If so, then we probably need a change to
> > UpdateMetadata as well. The only alternative I can think of would be to
> > represent the replica set in the Metadata request as the union of the
> > current and target replicas, but I can't think of any benefit to hiding
> > reassignments. Note that if we did this, we probably wouldn't need a
> > separate API to list reassignments.
> 
> I thought about this a bit... and I think on balance, you're right.  We 
> should keep this information together with the replica nodes, isr 
> nodes, and offline replicas, and that information is available in the 
> MetadataResponse. 
>  However, I do think in order to do this, we'll need a flag in the 
> MetadataRequest that specifiies "only show me reassigning partitions".  
> I'll add this.

I revisited this, and I think we should stick with the original proposal of having a separate ListPartitionReassignments API.  There really is no use case where the Producer or Consumer needs to know about a reassignment.  They should just be notified when the set of partitions changes, which doesn't require changes to MetadataRequest/Response.  The Admin client only cares if someone is managing the reassignment.  So adding this state to the MetadataResponse adds overhead for no real benefit.  In the common case where there is no ongoing reassignment, it would be 4 bytes per partition of extra overhead in the MetadataResponse.

In general, I think we have a problem of oversharing in the MetadataRequest/Response.  As we 10x or 100x the number of partitions we support, we'll need to get stricter about giving clients only the information they actually need, about the partitions they actually care about.  Reassignment state clearly falls in the category of state that isn't needed by clients (except very specialized rebalancing programs).

Another important consideration here is that someone managing an ongoing reassignment wants the most up-to-date information, which is to be found on the controller.  Therefore adding this state to listTopics or describeTopics, which could contact any node in the cluster, is sub-optimal.

Finally, adding this to listTopics or describeTopics feels like a warty API.  It's an extra boolean which interacts with other extra booleans like "show internal", etc. in weird ways.  I think a separate API is cleaner.

> 
> > 3. As replicas come into sync, they will join the ISR. Will we await all
> > target replicas joining the ISR before taking the replica out of the target
> > replicas set? Also, I assume that target replicas can still be elected as
> > leader?
> 
> We'll take a replica out of the target replicas set as soon as that 
> replica is in the ISR.  Let me clarify this in the KIP.
> 
> > 4. Probably useful to mention permissions for the new APIs.
> 
> Good point.  I think alterPartitionAssignments should require ALTER on 
> CLUSTER.  MetadataRequest permissions will be unchanged.

I added permission information.

best,
Colin

> 
> best,
> Colin
> 
> > 
> > Thanks,
> > Jason
> > 
> > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:
> > 
> > > +1 (binding)
> > > Looks great, and will be awesome to have this new capability.
> > >
> > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I'd like to start the vote for KIP-455: Create an Administrative API for
> > > > Replica Reassignment.  I think this KIP is important since it will unlock
> > > > many follow-on improvements to Kafka reassignment (see the "Future work"
> > > > section, plus a lot of the other discussions we've had recently about
> > > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > > direct access to ZK.
> > > >
> > > > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > > > Robert suggested, I removed the need to explicitly cancel a reassignment
> > > > for a partition before setting up a different reassignment for that
> > > > specific partition.  I also simplified the API a bit by adding a
> > > > PartitionReassignment class which is used by both the alter and list
> > > APIs.
> > > >
> > > > I modified the proposal so that we now deprecate the old znode-based API
> > > > rather than removing it completely.  That should give external
> > > rebalancing
> > > > tools some time to transition to the new API.
> > > >
> > > > To clarify a question Viktor asked, I added a note that the
> > > > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> > > to
> > > > contact the admin APIs.
> > > >
> > > > thanks,
> > > > Colin
> > > >
> > >
> > >
> > > --
> > > *Gwen Shapira*
> > > Product Manager | Confluent
> > > 650.450.2760 | @gwenshap
> > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > <http://www.confluent.io/blog>
> > >
> >
>
  

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
On Fri, May 10, 2019, at 17:34, Colin McCabe wrote:
> On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> > Hi Colin,
> > 
> > I think storing reassignment state at the partition level is the right move
> > and I also agree that replicas should understand that there is a
> > reassignment in progress. This makes KIP-352 a trivial follow-up for
> > example. The only doubt I have is whether the leader and isr znode is the
> > right place to store the target reassignment. It is a bit odd to keep the
> > target assignment in a separate place from the current assignment, right? I
> > assume the thinking is probably that although the current assignment should
> > probably be in the leader and isr znode as well, it is hard to move the
> > state in a compatible way. Is that right? But if we have no plan to remove
> > the assignment znode, do you see a downside to storing the target
> > assignment there as well?
> >
> 
> Hi Jason,
> 
> That's a good point -- it's probably better to keep the target 
> assignment in the same znode as the current assignment, for 
> consistency.  I'll change the KIP.

Hi Jason,

Thanks again for the review.

I took another look at this, and I think we should stick with the initial proposal of putting the reassignment state into /brokers/topics/[topic]/partitions/[partitionId]/state.  The reason is because we'll want to bump the leader epoch for the partition when changing the reassignment state, and the leader epoch resides in that znode anyway.  I agree there is some inconsistency here, but so be it: if we were to greenfield these zookeeper data structures, we might do it differently, but the proposed scheme will work fine and be extensible for the future.

> 
> > A few additional questions:
> > 
> > 1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
> > It's the current assignment we're altering, right?
> 
> That's fair.  AlterPartitionAssigments reads a little better, and I'll 
> change it to that.

+1.  I've changed the RPC and API name in the wiki.

> 
> > 2. Does this change affect the Metadata API? In other words, are clients
> > aware of reassignments? If so, then we probably need a change to
> > UpdateMetadata as well. The only alternative I can think of would be to
> > represent the replica set in the Metadata request as the union of the
> > current and target replicas, but I can't think of any benefit to hiding
> > reassignments. Note that if we did this, we probably wouldn't need a
> > separate API to list reassignments.
> 
> I thought about this a bit... and I think on balance, you're right.  We 
> should keep this information together with the replica nodes, isr 
> nodes, and offline replicas, and that information is available in the 
> MetadataResponse. 
>  However, I do think in order to do this, we'll need a flag in the 
> MetadataRequest that specifiies "only show me reassigning partitions".  
> I'll add this.

I revisited this, and I think we should stick with the original proposal of having a separate ListPartitionReassignments API.  There really is no use case where the Producer or Consumer needs to know about a reassignment.  They should just be notified when the set of partitions changes, which doesn't require changes to MetadataRequest/Response.  The Admin client only cares if someone is managing the reassignment.  So adding this state to the MetadataResponse adds overhead for no real benefit.  In the common case where there is no ongoing reassignment, it would be 4 bytes per partition of extra overhead in the MetadataResponse.

In general, I think we have a problem of oversharing in the MetadataRequest/Response.  As we 10x or 100x the number of partitions we support, we'll need to get stricter about giving clients only the information they actually need, about the partitions they actually care about.  Reassignment state clearly falls in the category of state that isn't needed by clients (except very specialized rebalancing programs).

Another important consideration here is that someone managing an ongoing reassignment wants the most up-to-date information, which is to be found on the controller.  Therefore adding this state to listTopics or describeTopics, which could contact any node in the cluster, is sub-optimal.

Finally, adding this to listTopics or describeTopics feels like a warty API.  It's an extra boolean which interacts with other extra booleans like "show internal", etc. in weird ways.  I think a separate API is cleaner.

> 
> > 3. As replicas come into sync, they will join the ISR. Will we await all
> > target replicas joining the ISR before taking the replica out of the target
> > replicas set? Also, I assume that target replicas can still be elected as
> > leader?
> 
> We'll take a replica out of the target replicas set as soon as that 
> replica is in the ISR.  Let me clarify this in the KIP.
> 
> > 4. Probably useful to mention permissions for the new APIs.
> 
> Good point.  I think alterPartitionAssignments should require ALTER on 
> CLUSTER.  MetadataRequest permissions will be unchanged.

I added permission information.

best,
Colin

> 
> best,
> Colin
> 
> > 
> > Thanks,
> > Jason
> > 
> > On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:
> > 
> > > +1 (binding)
> > > Looks great, and will be awesome to have this new capability.
> > >
> > > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I'd like to start the vote for KIP-455: Create an Administrative API for
> > > > Replica Reassignment.  I think this KIP is important since it will unlock
> > > > many follow-on improvements to Kafka reassignment (see the "Future work"
> > > > section, plus a lot of the other discussions we've had recently about
> > > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > > direct access to ZK.
> > > >
> > > > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > > > Robert suggested, I removed the need to explicitly cancel a reassignment
> > > > for a partition before setting up a different reassignment for that
> > > > specific partition.  I also simplified the API a bit by adding a
> > > > PartitionReassignment class which is used by both the alter and list
> > > APIs.
> > > >
> > > > I modified the proposal so that we now deprecate the old znode-based API
> > > > rather than removing it completely.  That should give external
> > > rebalancing
> > > > tools some time to transition to the new API.
> > > >
> > > > To clarify a question Viktor asked, I added a note that the
> > > > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> > > to
> > > > contact the admin APIs.
> > > >
> > > > thanks,
> > > > Colin
> > > >
> > >
> > >
> > > --
> > > *Gwen Shapira*
> > > Product Manager | Confluent
> > > 650.450.2760 | @gwenshap
> > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > <http://www.confluent.io/blog>
> > >
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Colin McCabe <cm...@apache.org>.
On Fri, May 10, 2019, at 16:43, Jason Gustafson wrote:
> Hi Colin,
> 
> I think storing reassignment state at the partition level is the right move
> and I also agree that replicas should understand that there is a
> reassignment in progress. This makes KIP-352 a trivial follow-up for
> example. The only doubt I have is whether the leader and isr znode is the
> right place to store the target reassignment. It is a bit odd to keep the
> target assignment in a separate place from the current assignment, right? I
> assume the thinking is probably that although the current assignment should
> probably be in the leader and isr znode as well, it is hard to move the
> state in a compatible way. Is that right? But if we have no plan to remove
> the assignment znode, do you see a downside to storing the target
> assignment there as well?
>

Hi Jason,

That's a good point -- it's probably better to keep the target assignment in the same znode as the current assignment, for consistency.  I'll change the KIP.

> A few additional questions:
> 
> 1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
> It's the current assignment we're altering, right?

That's fair.  AlterPartitionAssigments reads a little better, and I'll change it to that.

> 2. Does this change affect the Metadata API? In other words, are clients
> aware of reassignments? If so, then we probably need a change to
> UpdateMetadata as well. The only alternative I can think of would be to
> represent the replica set in the Metadata request as the union of the
> current and target replicas, but I can't think of any benefit to hiding
> reassignments. Note that if we did this, we probably wouldn't need a
> separate API to list reassignments.

I thought about this a bit... and I think on balance, you're right.  We should keep this information together with the replica nodes, isr nodes, and offline replicas, and that information is available in the MetadataResponse. 
 However, I do think in order to do this, we'll need a flag in the MetadataRequest that specifiies "only show me reassigning partitions".  I'll add this.

> 3. As replicas come into sync, they will join the ISR. Will we await all
> target replicas joining the ISR before taking the replica out of the target
> replicas set? Also, I assume that target replicas can still be elected as
> leader?

We'll take a replica out of the target replicas set as soon as that replica is in the ISR.  Let me clarify this in the KIP.

> 4. Probably useful to mention permissions for the new APIs.

Good point.  I think alterPartitionAssignments should require ALTER on CLUSTER.  MetadataRequest permissions will be unchanged.

best,
Colin

> 
> Thanks,
> Jason
> 
> On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:
> 
> > +1 (binding)
> > Looks great, and will be awesome to have this new capability.
> >
> > On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi all,
> > >
> > > I'd like to start the vote for KIP-455: Create an Administrative API for
> > > Replica Reassignment.  I think this KIP is important since it will unlock
> > > many follow-on improvements to Kafka reassignment (see the "Future work"
> > > section, plus a lot of the other discussions we've had recently about
> > > reassignment).  It also furthers the important KIP-4 goal of removing
> > > direct access to ZK.
> > >
> > > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > > Robert suggested, I removed the need to explicitly cancel a reassignment
> > > for a partition before setting up a different reassignment for that
> > > specific partition.  I also simplified the API a bit by adding a
> > > PartitionReassignment class which is used by both the alter and list
> > APIs.
> > >
> > > I modified the proposal so that we now deprecate the old znode-based API
> > > rather than removing it completely.  That should give external
> > rebalancing
> > > tools some time to transition to the new API.
> > >
> > > To clarify a question Viktor asked, I added a note that the
> > > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> > to
> > > contact the admin APIs.
> > >
> > > thanks,
> > > Colin
> > >
> >
> >
> > --
> > *Gwen Shapira*
> > Product Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > <http://www.confluent.io/blog>
> >
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Colin,

I think storing reassignment state at the partition level is the right move
and I also agree that replicas should understand that there is a
reassignment in progress. This makes KIP-352 a trivial follow-up for
example. The only doubt I have is whether the leader and isr znode is the
right place to store the target reassignment. It is a bit odd to keep the
target assignment in a separate place from the current assignment, right? I
assume the thinking is probably that although the current assignment should
probably be in the leader and isr znode as well, it is hard to move the
state in a compatible way. Is that right? But if we have no plan to remove
the assignment znode, do you see a downside to storing the target
assignment there as well?

A few additional questions:

1. Should `alterPartitionReassignments` be `alterPartitionAssignments`?
It's the current assignment we're altering, right?
2. Does this change affect the Metadata API? In other words, are clients
aware of reassignments? If so, then we probably need a change to
UpdateMetadata as well. The only alternative I can think of would be to
represent the replica set in the Metadata request as the union of the
current and target replicas, but I can't think of any benefit to hiding
reassignments. Note that if we did this, we probably wouldn't need a
separate API to list reassignments.
3. As replicas come into sync, they will join the ISR. Will we await all
target replicas joining the ISR before taking the replica out of the target
replicas set? Also, I assume that target replicas can still be elected as
leader?
4. Probably useful to mention permissions for the new APIs.

Thanks,
Jason

On Fri, May 10, 2019 at 9:30 AM Gwen Shapira <gw...@confluent.io> wrote:

> +1 (binding)
> Looks great, and will be awesome to have this new capability.
>
> On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:
>
> > Hi all,
> >
> > I'd like to start the vote for KIP-455: Create an Administrative API for
> > Replica Reassignment.  I think this KIP is important since it will unlock
> > many follow-on improvements to Kafka reassignment (see the "Future work"
> > section, plus a lot of the other discussions we've had recently about
> > reassignment).  It also furthers the important KIP-4 goal of removing
> > direct access to ZK.
> >
> > I made a few changes based on the discussion in the [DISCUSS] thread.  As
> > Robert suggested, I removed the need to explicitly cancel a reassignment
> > for a partition before setting up a different reassignment for that
> > specific partition.  I also simplified the API a bit by adding a
> > PartitionReassignment class which is used by both the alter and list
> APIs.
> >
> > I modified the proposal so that we now deprecate the old znode-based API
> > rather than removing it completely.  That should give external
> rebalancing
> > tools some time to transition to the new API.
> >
> > To clarify a question Viktor asked, I added a note that the
> > kafka-reassign-partitions.sh will now use a --bootstrap-server argument
> to
> > contact the admin APIs.
> >
> > thanks,
> > Colin
> >
>
>
> --
> *Gwen Shapira*
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> <http://www.confluent.io/blog>
>

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

Posted by Gwen Shapira <gw...@confluent.io>.
+1 (binding)
Looks great, and will be awesome to have this new capability.

On Wed, May 8, 2019 at 10:23 PM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> I'd like to start the vote for KIP-455: Create an Administrative API for
> Replica Reassignment.  I think this KIP is important since it will unlock
> many follow-on improvements to Kafka reassignment (see the "Future work"
> section, plus a lot of the other discussions we've had recently about
> reassignment).  It also furthers the important KIP-4 goal of removing
> direct access to ZK.
>
> I made a few changes based on the discussion in the [DISCUSS] thread.  As
> Robert suggested, I removed the need to explicitly cancel a reassignment
> for a partition before setting up a different reassignment for that
> specific partition.  I also simplified the API a bit by adding a
> PartitionReassignment class which is used by both the alter and list APIs.
>
> I modified the proposal so that we now deprecate the old znode-based API
> rather than removing it completely.  That should give external rebalancing
> tools some time to transition to the new API.
>
> To clarify a question Viktor asked, I added a note that the
> kafka-reassign-partitions.sh will now use a --bootstrap-server argument to
> contact the admin APIs.
>
> thanks,
> Colin
>


-- 
*Gwen Shapira*
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
<http://www.confluent.io/blog>