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

2020-03-26 Thread Colin McCabe
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=42=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  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 
> > > > 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
> > > > >  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
> 

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

2019-10-25 Thread Stanislav Kozlovski
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=42=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  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 
> > > 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
> > > >  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 wou

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

2019-08-28 Thread Colin McCabe
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  
> > 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 
> > >  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"  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 

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

2019-08-08 Thread Colin McCabe
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  
> 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 
> >  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"  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}
> > > > 

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

2019-08-08 Thread Koushik Chitta
Thanks Colin, George.   Can we restart the voting for this KIP.

Thanks,
Koushik 

-Original Message-
From: Colin McCabe  
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 
>  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"  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  
> > > Sent: Friday, August 2, 2019 10:04 AM
> > > To: dev 
> > > Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica 
> > > Reassignment
> > > 
> > > Hi, Colin,
> > > 
> > > First, since we are changing the format o

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

2019-08-07 Thread Colin McCabe
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 
>  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"  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  
> > > Sent: Friday, August 2, 2019 10:04 AM
> > > To: dev 
> > > 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

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

2019-08-07 Thread George Li
This email seemed to get lost in the dev email server.  Resending. 


On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li 
 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"  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  
> > Sent: Friday, August 2, 2019 10:04 AM
> > To: dev 
> > 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  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 c

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

2019-08-06 Thread Colin McCabe
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  
> Sent: Friday, August 2, 2019 10:04 AM
> To: dev 
> 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  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  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 

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

2019-08-06 Thread Koushik Chitta
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  
Sent: Friday, August 2, 2019 10:04 AM
To: dev 
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  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  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 t

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

2019-08-02 Thread Jun Rao
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  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  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
> > > 

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

2019-08-01 Thread Colin McCabe
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  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, 

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

2019-08-01 Thread Jun Rao
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  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 
> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > With three non-binding +1 votes from Viktor Somogyi-Vass, Robert
> > > Barrett,
> > > > > and George Li, and 3 binding +1 votes from 

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

2019-07-31 Thread Colin McCabe
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  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  > > > .invalid>
> > > > > wrote:
> > > > >
> > > > > >  +1 (non-binding)
> > > > > >
> > > > > >
> > > > > >
> > > > > > Thanks for addressing the comments.
> > > > > > George
> > > > > >
> > > > > > On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > > > g...@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 

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

2019-07-31 Thread Jun Rao
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 
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=36=35
>
> Thanks,
> Stanislav
>
> On Tue, Jul 30, 2019 at 10:18 PM Jun Rao  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  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  > > .invalid>
> > > > wrote:
> > > >
> > > > >  +1 (non-binding)
> > > > >
> > > > >
> > > > >
> > > > > Thanks for addressing the comments.
> > > > > George
> > > > >
> > > > > On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > > g...@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  >
> > > 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
> > > > > 

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

2019-07-31 Thread Stanislav Kozlovski
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=36=35

Thanks,
Stanislav

On Tue, Jul 30, 2019 at 10:18 PM Jun Rao  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  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  > .invalid>
> > > wrote:
> > >
> > > >  +1 (non-binding)
> > > >
> > > >
> > > >
> > > > Thanks for addressing the comments.
> > > > George
> > > >
> > > > On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > > g...@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 
> > > > wrote:
> > > > >
> > > > > +1 Thanks for the KIP. Really looking forward to this!
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe 
> > 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 

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

2019-07-30 Thread Jun Rao
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  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  .invalid>
> > wrote:
> >
> > >  +1 (non-binding)
> > >
> > >
> > >
> > > Thanks for addressing the comments.
> > > George
> > >
> > > On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > > g...@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 
> > > wrote:
> > > >
> > > > +1 Thanks for the KIP. Really looking forward to this!
> > > >
> > > > -Jason
> > > >
> > > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe 
> 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  >
> > > 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
> > > > > > > >  wrote:
> > > > > > > >
> > > > > > > >  Hi George,
> > > > > > > >
> > > > > > > > Yes, KIP-455 allows the 

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

2019-07-22 Thread Colin McCabe
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 
> wrote:
> 
> >  +1 (non-binding)
> >
> >
> >
> > Thanks for addressing the comments.
> > George
> >
> > On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> > g...@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 
> > wrote:
> > >
> > > +1 Thanks for the KIP. Really looking forward to this!
> > >
> > > -Jason
> > >
> > > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe  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 
> > 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
> > > > > > >  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
> > > > > > > >  wrote:
> > > > > > > >
> > > > > > > >  On Fri, 

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

2019-07-19 Thread Robert Barrett
+1 (non-binding). Thanks for the KIP!

On Thu, Jul 18, 2019 at 5:59 PM George Li 
wrote:

>  +1 (non-binding)
>
>
>
> Thanks for addressing the comments.
> George
>
> On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira <
> g...@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 
> wrote:
> >
> > +1 Thanks for the KIP. Really looking forward to this!
> >
> > -Jason
> >
> > On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe  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 
> 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
> > > > > >  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
> > > > > > >  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
> > 

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

2019-07-18 Thread George Li
 +1 (non-binding)



Thanks for addressing the comments. 
George

On Thursday, July 18, 2019, 05:03:58 PM PDT, Gwen Shapira 
 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  wrote:
>
> +1 Thanks for the KIP. Really looking forward to this!
>
> -Jason
>
> On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe  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  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
> > > > >  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
> > > > > >  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 

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

2019-07-18 Thread Gwen Shapira
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  wrote:
>
> +1 Thanks for the KIP. Really looking forward to this!
>
> -Jason
>
> On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe  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  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
> > > > >  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
> > > > > >  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 

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

2019-07-18 Thread Jason Gustafson
+1 Thanks for the KIP. Really looking forward to this!

-Jason

On Wed, Jul 17, 2019 at 1:41 PM Colin McCabe  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  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
> > > >  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
> > > > >  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 

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

2019-07-17 Thread Colin McCabe
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  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
> > >  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
> > > >  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, 

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

2019-07-17 Thread Stanislav Kozlovski
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  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
> >  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
> > >  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
> > > > 

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

2019-05-22 Thread Colin McCabe
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 
>  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 
> >  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 
> > 

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

2019-05-21 Thread George Li
 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 
 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 
>  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 

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

2019-05-16 Thread Colin McCabe
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 
>  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 

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

2019-05-14 Thread George Li
 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  
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 

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

2019-05-10 Thread Colin McCabe
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 

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

2019-05-10 Thread Colin McCabe
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  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  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  | blog
> > 
> >
>


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

2019-05-10 Thread Jason Gustafson
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  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  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  | blog
> 
>


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

2019-05-10 Thread Gwen Shapira
+1 (binding)
Looks great, and will be awesome to have this new capability.

On Wed, May 8, 2019 at 10:23 PM Colin McCabe  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  | blog



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

2019-05-09 Thread Robert Barrett
+1 (non-binding)

Thanks for the KIP, Colin!

On Thu, May 9, 2019 at 8:27 AM Colin McCabe  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  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

2019-05-09 Thread Colin McCabe
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  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

2019-05-09 Thread Viktor Somogyi-Vass
+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  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
>