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

2019-06-27 Thread Colin McCabe
On Thu, Jun 27, 2019, at 08:58, Jason Gustafson wrote:
> >
> > We'd remove nodes from targetReplicas just as soon as they entered the
> > ISR.  They would become regular replicas at that point.
> 
> 
> I think we can save a lot of back and forth by working through an example.
> Suppose we have the following initial state:
> 
> replicas: [1, 2, 3]
> isr: [1, 2, 3]
> targetReplicas: []
> 
> We want to reassign to [4, 5, 6]. My understanding is that leads to the
> following state:
> 
> replicas: [1, 2, 3]
> isr: [1, 2, 3]
> targetReplicas: [4, 5, 6]
> 
> Let's say replica 4 comes into the ISR first:
> 
> replicas: [1, 2, 3]
> isr: [1, 2, 3, 4]
> targetReplicas: [4, 5, 6]
> 
> What happens now?

The state transitions to:
replicas: [1, 2, 3, 4]
isr: [1, 2, 3, 4]
targetReplicas: [4, 5, 6]

The full sequence would be something like this:
R: [1, 2, 3], I: [1, 2, 3], T: [4, 5, 6]
R: [1, 2, 3], I: [1, 2, 3, 4], T: [4, 5, 6]
R: [1, 2, 3, 4], I: [1, 2, 3, 4], T: [4, 5, 6]
R: [1, 2, 3, 4], I: [1, 2, 3, 4, 5], T: [4, 5, 6]
R: [1, 2, 3, 4, 5], I: [1, 2, 3, 4, 5], T: [4, 5, 6]
R: [1, 2, 3, 4, 5], I: [1, 2, 3, 4, 5, 6], T: [4, 5, 6]
R: [1, 2, 3, 4, 5, 6], I: [1, 2, 3, 4, 5, 6], T: [4, 5, 6]
R: [4, 5, 6], I: [4, 5, 6], T: null

Here's another example:
R: [1, 2, 3], I: [1, 2, 3], T: [2, 3, 4]
R: [1, 2, 3], I: [1, 2, 3, 4], T: [2, 3, 4]
R: [1, 2, 3, 4], I: [1, 2, 3, 4], T: [2, 3, 4]
R: [2, 3, 4], I: [2, 3, 4], T: null

Basically, "target replicas" represents what you want the partition assignment 
to be.  "Replicas" represents what it currently is.  In other words, "target 
replicas" has the role that was formerly played by the assignment shown in 
/admin/partition_reassignments.

The main thing that is different is that we don't want to automatically add all 
the target replicas to the normal replica set.  The reason why we don't want to 
do this is because it removes the information we need for cancellation: namely, 
whether each replica was one of the original ones, or one of the ones we're 
reassigning to.  A secondary reason is because nothing useful will come of 
telling users they have a bunch of replicas that don't really exist.  We may 
not have even started copying the first byte over to a target replica, so it 
doesn't make sense to treat it like a normal replica that's not in the ISR 
because it's lagging slightly.

best,
Colin


> 
> (Sorry if I'm being dense, it's just not clear to me exactly what the
> expected transitions are from here.)
> 
> 
> Thanks,
> Jason
> 
> 
> On Wed, Jun 26, 2019 at 2:12 PM Colin McCabe  wrote:
> 
> > On Wed, Jun 26, 2019, at 12:02, Jason Gustafson wrote:
> > > Hi Colin,
> > >
> > > Responses below and another question:
> > >
> > > > I guess the thought process here is that most reassignment tools want
> > to
> > > > know about all the reassignments that are going on.  If you don't know
> > all
> > > > the pending reassignments, then it's hard to say whether adding a new
> > one
> > > > is a good idea, or cancelling an existing one.  So I guess I can't
> > think of
> > > > a case where a reassignment tool would want a partial set rather than
> > the
> > > > full one.
> > >
> > > UIs often have "drill into" options. If there is a large ongoing
> > > reassignment, I can see wanting to limit the scope to a specific topic.
> > Any
> > > downside that I'm missing?
> > >
> >
> > We could add a mode that only lists a given set of partitions.  To be
> > consistent with how we handle topics, that could be a separate "describe"
> > method.  I don't think there's any downside, except some extra code to
> > write.
> >
> > > > Good question.  It will be the current behavior.  Basically, we
> > immediately
> > > > try to replicate to all the targetReplicas.  As they enter the ISR,
> > they
> > > > leave "targetReplicas" and enter "replicas."  Making this more
> > incremental
> > > > would be a good follow-on improvement.
> > >
> > > The current behavior is to wait until all target replicas are in the ISR
> > > before making a change. Are you saying that we will keep this behavior?
> >
> > We'd remove nodes from targetReplicas just as soon as they entered the
> > ISR.  They would become regular replicas at that point.
> >
> > >
> > > > When the /admin/reassign_partitions znode is empty, we'll listen for
> > > > updates.  When an update is made, we'll treat it like a call to
> > > > AlterPartitionReassignments.  Whenever we have zero pending
> > reassignments,
> > > > we'll delete the /admin/reassign_partitions znode.  If the znode
> > already
> > > > exists, we don't listen on it (same as now).
> > >
> > >
> > > So just to be a bit more explicit, what you are saying is that we will
> > keep
> > > the reassignment state under /admin/reassign_partitions as we do
> > currently,
> > > but we will update the target_replicas field in /partition/state
> > following
> > > this new logic. Then as soon as the current replica set matches the
> > target
> > > assignment, we will remove the 

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

2019-06-27 Thread Jason Gustafson
>
> We'd remove nodes from targetReplicas just as soon as they entered the
> ISR.  They would become regular replicas at that point.


I think we can save a lot of back and forth by working through an example.
Suppose we have the following initial state:

replicas: [1, 2, 3]
isr: [1, 2, 3]
targetReplicas: []

We want to reassign to [4, 5, 6]. My understanding is that leads to the
following state:

replicas: [1, 2, 3]
isr: [1, 2, 3]
targetReplicas: [4, 5, 6]

Let's say replica 4 comes into the ISR first:

replicas: [1, 2, 3]
isr: [1, 2, 3, 4]
targetReplicas: [4, 5, 6]

What happens now?

(Sorry if I'm being dense, it's just not clear to me exactly what the
expected transitions are from here.)


Thanks,
Jason


On Wed, Jun 26, 2019 at 2:12 PM Colin McCabe  wrote:

> On Wed, Jun 26, 2019, at 12:02, Jason Gustafson wrote:
> > Hi Colin,
> >
> > Responses below and another question:
> >
> > > I guess the thought process here is that most reassignment tools want
> to
> > > know about all the reassignments that are going on.  If you don't know
> all
> > > the pending reassignments, then it's hard to say whether adding a new
> one
> > > is a good idea, or cancelling an existing one.  So I guess I can't
> think of
> > > a case where a reassignment tool would want a partial set rather than
> the
> > > full one.
> >
> > UIs often have "drill into" options. If there is a large ongoing
> > reassignment, I can see wanting to limit the scope to a specific topic.
> Any
> > downside that I'm missing?
> >
>
> We could add a mode that only lists a given set of partitions.  To be
> consistent with how we handle topics, that could be a separate "describe"
> method.  I don't think there's any downside, except some extra code to
> write.
>
> > > Good question.  It will be the current behavior.  Basically, we
> immediately
> > > try to replicate to all the targetReplicas.  As they enter the ISR,
> they
> > > leave "targetReplicas" and enter "replicas."  Making this more
> incremental
> > > would be a good follow-on improvement.
> >
> > The current behavior is to wait until all target replicas are in the ISR
> > before making a change. Are you saying that we will keep this behavior?
>
> We'd remove nodes from targetReplicas just as soon as they entered the
> ISR.  They would become regular replicas at that point.
>
> >
> > > When the /admin/reassign_partitions znode is empty, we'll listen for
> > > updates.  When an update is made, we'll treat it like a call to
> > > AlterPartitionReassignments.  Whenever we have zero pending
> reassignments,
> > > we'll delete the /admin/reassign_partitions znode.  If the znode
> already
> > > exists, we don't listen on it (same as now).
> >
> >
> > So just to be a bit more explicit, what you are saying is that we will
> keep
> > the reassignment state under /admin/reassign_partitions as we do
> currently,
> > but we will update the target_replicas field in /partition/state
> following
> > this new logic. Then as soon as the current replica set matches the
> target
> > assignment, we will remove the /admin/reassign_partitions znode. Right?
>
> One clarification: I wasn't proposing that the controller should write to
> /admin/reassign_partitions.  We will just remove the znode when we
> transition to having no ongoing reassignments.  There's no guarantee that
> what is in the znode reflects the current reassignments that are going on.
> The only thing you can know is that if the znode exists, there is at least
> one reassignment going on.  But if someone makes a new reassignment with
> the AlterPartitionReassignments API, it won't appear in the znode.
>
> Another thing to note is that if the znode exists and you overwrite it,
> your updates will be ignored.  This matches the current behavior of this
> znode.  Apparently some applications don't know about this behavior and try
> to update the znode while a reassignment is going on, but it has no
> effect-- other than making what is in ZK misleading if someone checks.
> This is, again, existing behavior :(
>
> It's not a good API by any means.  For example, what if someone else
> overwrites your znode update before the controller has a chance to read
> it?  Unfortunate, but there's no really good way to fix this without
> transitioning away from direct ZooKeeper access.  We'll transition the
> command line tools immediately, but there will be some external management
> tools that will lag a bit.
>
> >
> > Actually I'm still a bit confused about one aspect of this proposal. You
> > are suggesting that we should leave the reassignment out of the Metadata.
> > That is fine, but what does that mean as far as the consistency of the
> > metadata we expose to clients while a reassignment is active? Currently
> the
> > Metadata includes the following:
> >
> > 1. Current leader
> > 2. Current ISR
> > 3. Current assigned replicas
> >
> > Can you explain how the reassignment will affect this state?  As the
> target
> > replicas are coming into sync, they will 

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

2019-06-27 Thread Viktor Somogyi-Vass
Hey Colin,

So in my understanding this is how the controller handles a reassignment in
a simple scenario:
1. When an AlterPartitionReassignmentRequest arrives it updates the
partitions' ZK data in
/brokers/topics/[topic]/partitions/[partitionId]/state with targetReplicas
2. Sends out LeaderAndIsr requests to the reassigned partitions to update
their targetReplica set
3. Starts the new replicas
4. Waits until a new replica comes in sync, mark them online
5. Sends out an UpdateMetadata request to the replicas that the new
partitions became ISR
6. Updates the partition state in ZK by removing the new ISR from the
targetReplica set
7. Sends out a LeaderAndIsr request with the targetReplicas not containing
the new ISR. This request may contain a change in leadership info if leader
change is needed
8. If the reassigned replica has to be deleted, then take them offline and
then non-existent
9. Send out an UpdateMetadata request with the new ISR information

Is this correct? Let me know if I missed something, I just wanted to
assemble the picture a little bit. I think it would be useful to add this
(or the corrected version if needed :) ) to the KIP with an example maybe
as Jason suggested. Would help a lot in understandability.

Regards,
Viktor

On Wed, Jun 26, 2019 at 11:12 PM Colin McCabe  wrote:

> On Wed, Jun 26, 2019, at 12:02, Jason Gustafson wrote:
> > Hi Colin,
> >
> > Responses below and another question:
> >
> > > I guess the thought process here is that most reassignment tools want
> to
> > > know about all the reassignments that are going on.  If you don't know
> all
> > > the pending reassignments, then it's hard to say whether adding a new
> one
> > > is a good idea, or cancelling an existing one.  So I guess I can't
> think of
> > > a case where a reassignment tool would want a partial set rather than
> the
> > > full one.
> >
> > UIs often have "drill into" options. If there is a large ongoing
> > reassignment, I can see wanting to limit the scope to a specific topic.
> Any
> > downside that I'm missing?
> >
>
> We could add a mode that only lists a given set of partitions.  To be
> consistent with how we handle topics, that could be a separate "describe"
> method.  I don't think there's any downside, except some extra code to
> write.
>
> > > Good question.  It will be the current behavior.  Basically, we
> immediately
> > > try to replicate to all the targetReplicas.  As they enter the ISR,
> they
> > > leave "targetReplicas" and enter "replicas."  Making this more
> incremental
> > > would be a good follow-on improvement.
> >
> > The current behavior is to wait until all target replicas are in the ISR
> > before making a change. Are you saying that we will keep this behavior?
>
> We'd remove nodes from targetReplicas just as soon as they entered the
> ISR.  They would become regular replicas at that point.
>
> >
> > > When the /admin/reassign_partitions znode is empty, we'll listen for
> > > updates.  When an update is made, we'll treat it like a call to
> > > AlterPartitionReassignments.  Whenever we have zero pending
> reassignments,
> > > we'll delete the /admin/reassign_partitions znode.  If the znode
> already
> > > exists, we don't listen on it (same as now).
> >
> >
> > So just to be a bit more explicit, what you are saying is that we will
> keep
> > the reassignment state under /admin/reassign_partitions as we do
> currently,
> > but we will update the target_replicas field in /partition/state
> following
> > this new logic. Then as soon as the current replica set matches the
> target
> > assignment, we will remove the /admin/reassign_partitions znode. Right?
>
> One clarification: I wasn't proposing that the controller should write to
> /admin/reassign_partitions.  We will just remove the znode when we
> transition to having no ongoing reassignments.  There's no guarantee that
> what is in the znode reflects the current reassignments that are going on.
> The only thing you can know is that if the znode exists, there is at least
> one reassignment going on.  But if someone makes a new reassignment with
> the AlterPartitionReassignments API, it won't appear in the znode.
>
> Another thing to note is that if the znode exists and you overwrite it,
> your updates will be ignored.  This matches the current behavior of this
> znode.  Apparently some applications don't know about this behavior and try
> to update the znode while a reassignment is going on, but it has no
> effect-- other than making what is in ZK misleading if someone checks.
> This is, again, existing behavior :(
>
> It's not a good API by any means.  For example, what if someone else
> overwrites your znode update before the controller has a chance to read
> it?  Unfortunate, but there's no really good way to fix this without
> transitioning away from direct ZooKeeper access.  We'll transition the
> command line tools immediately, but there will be some external management
> tools that will lag a bit.
>
> >
> > Actually I'm 

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

2019-06-26 Thread Colin McCabe
On Wed, Jun 26, 2019, at 12:02, Jason Gustafson wrote:
> Hi Colin,
> 
> Responses below and another question:
> 
> > I guess the thought process here is that most reassignment tools want to
> > know about all the reassignments that are going on.  If you don't know all
> > the pending reassignments, then it's hard to say whether adding a new one
> > is a good idea, or cancelling an existing one.  So I guess I can't think of
> > a case where a reassignment tool would want a partial set rather than the
> > full one.
>
> UIs often have "drill into" options. If there is a large ongoing
> reassignment, I can see wanting to limit the scope to a specific topic. Any
> downside that I'm missing?
>

We could add a mode that only lists a given set of partitions.  To be 
consistent with how we handle topics, that could be a separate "describe" 
method.  I don't think there's any downside, except some extra code to write.

> > Good question.  It will be the current behavior.  Basically, we immediately
> > try to replicate to all the targetReplicas.  As they enter the ISR, they
> > leave "targetReplicas" and enter "replicas."  Making this more incremental
> > would be a good follow-on improvement.
> 
> The current behavior is to wait until all target replicas are in the ISR
> before making a change. Are you saying that we will keep this behavior?

We'd remove nodes from targetReplicas just as soon as they entered the ISR.  
They would become regular replicas at that point.

> 
> > When the /admin/reassign_partitions znode is empty, we'll listen for
> > updates.  When an update is made, we'll treat it like a call to
> > AlterPartitionReassignments.  Whenever we have zero pending reassignments,
> > we'll delete the /admin/reassign_partitions znode.  If the znode already
> > exists, we don't listen on it (same as now).
> 
> 
> So just to be a bit more explicit, what you are saying is that we will keep
> the reassignment state under /admin/reassign_partitions as we do currently,
> but we will update the target_replicas field in /partition/state following
> this new logic. Then as soon as the current replica set matches the target
> assignment, we will remove the /admin/reassign_partitions znode. Right?

One clarification: I wasn't proposing that the controller should write to 
/admin/reassign_partitions.  We will just remove the znode when we transition 
to having no ongoing reassignments.  There's no guarantee that what is in the 
znode reflects the current reassignments that are going on.  The only thing you 
can know is that if the znode exists, there is at least one reassignment going 
on.  But if someone makes a new reassignment with the 
AlterPartitionReassignments API, it won't appear in the znode.

Another thing to note is that if the znode exists and you overwrite it, your 
updates will be ignored.  This matches the current behavior of this znode.  
Apparently some applications don't know about this behavior and try to update 
the znode while a reassignment is going on, but it has no effect-- other than 
making what is in ZK misleading if someone checks.  This is, again, existing 
behavior :(

It's not a good API by any means.  For example, what if someone else overwrites 
your znode update before the controller has a chance to read it?  Unfortunate, 
but there's no really good way to fix this without transitioning away from 
direct ZooKeeper access.  We'll transition the command line tools immediately, 
but there will be some external management tools that will lag a bit.

> 
> Actually I'm still a bit confused about one aspect of this proposal. You
> are suggesting that we should leave the reassignment out of the Metadata.
> That is fine, but what does that mean as far as the consistency of the
> metadata we expose to clients while a reassignment is active? Currently the
> Metadata includes the following:
> 
> 1. Current leader
> 2. Current ISR
> 3. Current assigned replicas
> 
> Can you explain how the reassignment will affect this state?  As the target
> replicas are coming into sync, they will be added to the ISR. We wouldn't
> want it to be the case that the ISR includes replicas which are not
> currently assigned. So is the replica set the union of the current and
> target replicas?

I think we should leave the targetReplicas out of the normal replica set in the 
client metadata.  The client code that is checking the replica set shouldn't be 
treating target replicas as if they were regular replicas.   For example, you 
wouldn't look at a partition with 3 replicas and 2 target replicas, and an ISR 
of size 3, and think that it was under-replicated.

I agree that there is kind of an awkward edge case right when a targetReplica 
enters the ISR.  Since the replica set and the ISR reside in two separate 
znodes, they can't be updated atomically (well, maybe they could, but I don't 
think we will in practice).  When the controller becomes aware that the ISR has 
changed for the reassigning partition, it will 

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

2019-06-26 Thread Jason Gustafson
Hi Colin,

Responses below and another question:

I guess the thought process here is that most reassignment tools want to
> know about all the reassignments that are going on.  If you don't know all
> the pending reassignments, then it's hard to say whether adding a new one
> is a good idea, or cancelling an existing one.  So I guess I can't think of
> a case where a reassignment tool would want a partial set rather than the
> full one.


UIs often have "drill into" options. If there is a large ongoing
reassignment, I can see wanting to limit the scope to a specific topic. Any
downside that I'm missing?

Good question.  It will be the current behavior.  Basically, we immediately
> try to replicate to all the targetReplicas.  As they enter the ISR, they
> leave "targetReplicas" and enter "replicas."  Making this more incremental
> would be a good follow-on improvement.


The current behavior is to wait until all target replicas are in the ISR
before making a change. Are you saying that we will keep this behavior?

When the /admin/reassign_partitions znode is empty, we'll listen for
> updates.  When an update is made, we'll treat it like a call to
> AlterPartitionReassignments.  Whenever we have zero pending reassignments,
> we'll delete the /admin/reassign_partitions znode.  If the znode already
> exists, we don't listen on it (same as now).


So just to be a bit more explicit, what you are saying is that we will keep
the reassignment state under /admin/reassign_partitions as we do currently,
but we will update the target_replicas field in /partition/state following
this new logic. Then as soon as the current replica set matches the target
assignment, we will remove the /admin/reassign_partitions znode. Right?


Actually I'm still a bit confused about one aspect of this proposal. You
are suggesting that we should leave the reassignment out of the Metadata.
That is fine, but what does that mean as far as the consistency of the
metadata we expose to clients while a reassignment is active? Currently the
Metadata includes the following:

1. Current leader
2. Current ISR
3. Current assigned replicas

Can you explain how the reassignment will affect this state?  As the target
replicas are coming into sync, they will be added to the ISR. We wouldn't
want it to be the case that the ISR includes replicas which are not
currently assigned. So is the replica set the union of the current and
target replicas?

Thanks,
Jason




On Wed, Jun 26, 2019 at 10:42 AM Colin McCabe  wrote:

> On Tue, Jun 25, 2019, at 18:37, Jason Gustafson wrote:
> > Hi Colin,
> >
> > Took another pass on the KIP. Looks good overall. A few questions below:
> >
> > 1. I wasn't clear why `currentReplicas` is an optional field. Wouldn't we
> > always have a current set of replicas?
>
> Good point.  When I wrote that I was trying to use the same structure for
> both requesting a new reassignment, and describing an existing one.  I just
> posted a new version which uses separate structures for these.
> (CurrentReplicas is only relevant when you are listing an existing
> assignment.)
>
> > 2. Seems the only option is to list all active partition reassignments? I
> > think we have tended to regret these APIs. At least should we be able to
> > specify a subset of topics or partitions perhaps?
>
> I guess the thought process here is that most reassignment tools want to
> know about all the reassignments that are going on.  If you don't know all
> the pending reassignments, then it's hard to say whether adding a new one
> is a good idea, or cancelling an existing one.  So I guess I can't think of
> a case where a reassignment tool would want a partial set rather than the
> full one.
>
> It is kind of unfortunate to be passing all this information to some
> external process, but that's kind of inherent in the idea of reassignment
> as separate from the controller.  There's normally only one or zero
> processes doing and monitoring reassignments, so it's not as bad as
> thousands of clients sending full metadata requests.  It's probably OK?
>
> >
> > 3. Can you elaborate a bit on the handling of /admin/reassign_partitions?
> > Does this alter the target_replicas of the leader and ISR znode?
>
> When the /admin/reassign_partitions znode is empty, we'll listen for
> updates.  When an update is made, we'll treat it like a call to
> AlterPartitionReassignments.  Whenever we have zero pending reassignments,
> we'll delete the /admin/reassign_partitions znode.  If the znode already
> exists, we don't listen on it (same as now).
>
> One thing to note here is that if you upgrade from a pre-KIP-455 version
> to a KIP-455 version, and you invoke AlterPartitionReassignments before the
> cluster roll is finished, the new controller won't write the reassignment
> to /admin/reassign_partitions.  So if the controller node bounces to an
> un-upgraded broker during the roll, the reassignment will halt for that
> time period.  This is OK because a cluster roll should be a 

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

2019-06-26 Thread Colin McCabe
On Tue, Jun 25, 2019, at 18:37, Jason Gustafson wrote:
> Hi Colin,
> 
> Took another pass on the KIP. Looks good overall. A few questions below:
> 
> 1. I wasn't clear why `currentReplicas` is an optional field. Wouldn't we
> always have a current set of replicas?

Good point.  When I wrote that I was trying to use the same structure for both 
requesting a new reassignment, and describing an existing one.  I just posted a 
new version which uses separate structures for these.  (CurrentReplicas is only 
relevant when you are listing an existing assignment.)

> 2. Seems the only option is to list all active partition reassignments? I
> think we have tended to regret these APIs. At least should we be able to
> specify a subset of topics or partitions perhaps?

I guess the thought process here is that most reassignment tools want to know 
about all the reassignments that are going on.  If you don't know all the 
pending reassignments, then it's hard to say whether adding a new one is a good 
idea, or cancelling an existing one.  So I guess I can't think of a case where 
a reassignment tool would want a partial set rather than the full one.

It is kind of unfortunate to be passing all this information to some external 
process, but that's kind of inherent in the idea of reassignment as separate 
from the controller.  There's normally only one or zero processes doing and 
monitoring reassignments, so it's not as bad as thousands of clients sending 
full metadata requests.  It's probably OK?

> 
> 3. Can you elaborate a bit on the handling of /admin/reassign_partitions?
> Does this alter the target_replicas of the leader and ISR znode?

When the /admin/reassign_partitions znode is empty, we'll listen for updates.  
When an update is made, we'll treat it like a call to 
AlterPartitionReassignments.  Whenever we have zero pending reassignments, 
we'll delete the /admin/reassign_partitions znode.  If the znode already 
exists, we don't listen on it (same as now).

One thing to note here is that if you upgrade from a pre-KIP-455 version to a 
KIP-455 version, and you invoke AlterPartitionReassignments before the cluster 
roll is finished, the new controller won't write the reassignment to 
/admin/reassign_partitions.  So if the controller node bounces to an 
un-upgraded broker during the roll, the reassignment will halt for that time 
period.  This is OK because a cluster roll should be a relatively short amount 
of time-- and also, probably not a time you want to be doing reassignments 
anyway because of the extra load on the cluster.

> 
> 4. I think it would be helpful to provide an example of the rebalance
> process for a given partition. Specifically I am wondering whether the
> replica set is updated incrementally or if we follow the current behavior.
> Possibly some decisions can be deferred to implementation, but it would be
> helpful to work through a case of changing the replication factor just to
> make sure there are reasonable options.

Good question.  It will be the current behavior.  Basically, we immediately try 
to replicate to all the targetReplicas.  As they enter the ISR, they leave 
"targetReplicas" and enter "replicas."  Making this more incremental would be a 
good follow-on improvement.

> 
> 5. Are we changing the semantics of the URP and UnderMinIsr metrics in this
> KIP or in a follow-up?

In a follow-up.

> 
> 6. We have both "TargetBrokers" and "PendingReplicas" as names in the
> proposal. Perhaps we should try to be consistent?

Fixed.  I set it to TargetReplicas for now

> 
> 7. I am not sure specifying `targetReplicas` as empty is the clearest way
> to cancel a reassignment. Whether we implement it this way or not in the
> protocol is a separate issue, but perhaps we should have an explicit
> `cancelReassignment` method in AdminClient?

I changed the API slightly to take an Optional.  Does 
that seem cleaner?

best,
Colin

> 
> Thanks,
> Jason
> 
> 
> 
> 
> On Wed, Jun 19, 2019 at 3:36 AM Stanislav Kozlovski 
> wrote:
> 
> > Hey there Colin,
> >
> > Thanks for the work on this KIP. It is a much-needed improvement and I'm
> > excited to see it. Sorry for coming in so late to the discussion, I have
> > one question to better understand the change and a small suggestion.
> >
> > I see we allow reassignment cancellation at the partition level - what is
> > the motivation behind that? I think that having the back-end structures
> > support it is a good choice since it allows us more flexibility in the
> > future but what are the reasons for allowing a user to cancel at a
> > partition level? I think allowing it might let users shoot themselves in
> > the foot easier and make tools harder to implement (needing to guard
> > against it).
> >
> > In all regards, what do you think about an ease of use improvement where we
> > allow a user to cancel all reassignments for a topic without specifying its
> > partitions? Essentially, we could cancel all reassignments for a topic if
> > the 

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

2019-06-25 Thread Colin McCabe
On Wed, Jun 19, 2019, at 03:36, Stanislav Kozlovski wrote:
> Hey there Colin,
> 
> Thanks for the work on this KIP. It is a much-needed improvement and I'm
> excited to see it. Sorry for coming in so late to the discussion, I have
> one question to better understand the change and a small suggestion.
> 
> I see we allow reassignment cancellation at the partition level - what is
> the motivation behind that? I think that having the back-end structures
> support it is a good choice since it allows us more flexibility in the
> future but what are the reasons for allowing a user to cancel at a
> partition level? I think allowing it might let users shoot themselves in
> the foot easier and make tools harder to implement (needing to guard
> against it).

Hi Stanislav,

Thanks for taking a look.

I'm not sure I follow the comment about cancellation.  Why would we need to 
guard against someone cancelling a reassignment of a single partition?  Why 
would cancelling a single partition's reassignment be "shooting yourself in the 
foot"?

> In all regards, what do you think about an ease of use improvement where we
> allow a user to cancel all reassignments for a topic without specifying its
> partitions? Essentially, we could cancel all reassignments for a topic if
> the Partitions field in AlterPartitionAssignmentsRequest is null.

I'm not sure why we would want to cancel all reassignments for a particular 
topic.  In general, I would expect reassignments to be cancelled if the 
situation on a broker changed, and it was overloaded instead of underloaded.  
Or, some reassignments might be cancelled if they created more overhead for 
users of the system than anticipated.  In both of these cases, topics are not 
really relevant.

After all, the partitions for a particular topic are probably spread across the 
whole system.  Topics are a useful administrative concept, but not really that 
relevant to the world of partition reassignment (or maybe I'm missing 
something?)

best,
Colin


> 
> Best,
> Stanislav
> 
> On Mon, May 6, 2019 at 5:42 PM Colin McCabe  wrote:
> 
> > On Mon, May 6, 2019, at 07:39, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > A quick comment.
> > >
> > > On Sat, May 4, 2019 at 11:18 PM Colin McCabe  wrote:
> > >
> > > > The big advantage of doing batching on the controller is that the
> > > > controller has more information about what is going on in the
> > cluster.  So
> > > > it can schedule reassignments in a more optimal way.  For instance, it
> > can
> > > > schedule reassignments so that the load is distributed evenly across
> > > > nodes.  This advantage is lost if we have to adhere to a rigid ordering
> > > > that is set up in advance.  We don't know exactly when anything will
> > > > complete in any case.  Just because one partition reassignment was
> > started
> > > > before another doesn't mean it will finish before another.
> > >
> > >
> > > This is not quite true, right? The Controller doesn't know about
> > partition
> > > sizes, throughput per partition and other such information that external
> > > tools like Cruise Control track.
> >
> > Hi Ismael,
> >
> > That's a good point, and one I should have included.
> >
> > I guess when I think about "do batching in the controller" versus "do
> > batching in an external system" I tend to think about the information the
> > controller could theoretically collect, rather than what it actually does
> > :)  But certainly, adding this information to the controller would be a
> > significant change, and maybe one we don't want to do if the external
> > systems work well enough.
> >
> > Thinking about this a little bit more, I can see three advantages to
> > controller-side batching.  Firstly, doing batching in the controller saves
> > memory because we don't use a separate JVM, and don't duplicate the
> > in-memory map of all the partitions.  Secondly, the information we're
> > acting on would also be more up-to-date.  (I'm not sure how important this
> > would be.)  Finally, it's one less thing to deploy.  I don't know if those
> > are really enough to motivate switching now, but in a greenfield system I
> > would probably choose controller-side rebalancing.
> >
> > In any case, this KIP is orthogonal to controller-side rebalancing versus
> > external rebalancing.  That's why the KIP states that we will continue to
> > perform all the given partition rebalances immediately.  I was just
> > responding to the idea that maybe we should have an "ordering" of
> > rebalancing partitions.  I don't think we want that, for controller-side
> > rebalancing or externally batched rebalancing.
> >
> > best,
> > Colin
> >
> 
> 
> -- 
> Best,
> Stanislav
>


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

2019-06-25 Thread Jason Gustafson
Hi Colin,

Took another pass on the KIP. Looks good overall. A few questions below:

1. I wasn't clear why `currentReplicas` is an optional field. Wouldn't we
always have a current set of replicas?

2. Seems the only option is to list all active partition reassignments? I
think we have tended to regret these APIs. At least should we be able to
specify a subset of topics or partitions perhaps?

3. Can you elaborate a bit on the handling of /admin/reassign_partitions?
Does this alter the target_replicas of the leader and ISR znode?

4. I think it would be helpful to provide an example of the rebalance
process for a given partition. Specifically I am wondering whether the
replica set is updated incrementally or if we follow the current behavior.
Possibly some decisions can be deferred to implementation, but it would be
helpful to work through a case of changing the replication factor just to
make sure there are reasonable options.

5. Are we changing the semantics of the URP and UnderMinIsr metrics in this
KIP or in a follow-up?

6. We have both "TargetBrokers" and "PendingReplicas" as names in the
proposal. Perhaps we should try to be consistent?

7. I am not sure specifying `targetReplicas` as empty is the clearest way
to cancel a reassignment. Whether we implement it this way or not in the
protocol is a separate issue, but perhaps we should have an explicit
`cancelReassignment` method in AdminClient?

Thanks,
Jason




On Wed, Jun 19, 2019 at 3:36 AM Stanislav Kozlovski 
wrote:

> Hey there Colin,
>
> Thanks for the work on this KIP. It is a much-needed improvement and I'm
> excited to see it. Sorry for coming in so late to the discussion, I have
> one question to better understand the change and a small suggestion.
>
> I see we allow reassignment cancellation at the partition level - what is
> the motivation behind that? I think that having the back-end structures
> support it is a good choice since it allows us more flexibility in the
> future but what are the reasons for allowing a user to cancel at a
> partition level? I think allowing it might let users shoot themselves in
> the foot easier and make tools harder to implement (needing to guard
> against it).
>
> In all regards, what do you think about an ease of use improvement where we
> allow a user to cancel all reassignments for a topic without specifying its
> partitions? Essentially, we could cancel all reassignments for a topic if
> the Partitions field in AlterPartitionAssignmentsRequest is null.
>
> Best,
> Stanislav
>
> On Mon, May 6, 2019 at 5:42 PM Colin McCabe  wrote:
>
> > On Mon, May 6, 2019, at 07:39, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > A quick comment.
> > >
> > > On Sat, May 4, 2019 at 11:18 PM Colin McCabe 
> wrote:
> > >
> > > > The big advantage of doing batching on the controller is that the
> > > > controller has more information about what is going on in the
> > cluster.  So
> > > > it can schedule reassignments in a more optimal way.  For instance,
> it
> > can
> > > > schedule reassignments so that the load is distributed evenly across
> > > > nodes.  This advantage is lost if we have to adhere to a rigid
> ordering
> > > > that is set up in advance.  We don't know exactly when anything will
> > > > complete in any case.  Just because one partition reassignment was
> > started
> > > > before another doesn't mean it will finish before another.
> > >
> > >
> > > This is not quite true, right? The Controller doesn't know about
> > partition
> > > sizes, throughput per partition and other such information that
> external
> > > tools like Cruise Control track.
> >
> > Hi Ismael,
> >
> > That's a good point, and one I should have included.
> >
> > I guess when I think about "do batching in the controller" versus "do
> > batching in an external system" I tend to think about the information the
> > controller could theoretically collect, rather than what it actually does
> > :)  But certainly, adding this information to the controller would be a
> > significant change, and maybe one we don't want to do if the external
> > systems work well enough.
> >
> > Thinking about this a little bit more, I can see three advantages to
> > controller-side batching.  Firstly, doing batching in the controller
> saves
> > memory because we don't use a separate JVM, and don't duplicate the
> > in-memory map of all the partitions.  Secondly, the information we're
> > acting on would also be more up-to-date.  (I'm not sure how important
> this
> > would be.)  Finally, it's one less thing to deploy.  I don't know if
> those
> > are really enough to motivate switching now, but in a greenfield system I
> > would probably choose controller-side rebalancing.
> >
> > In any case, this KIP is orthogonal to controller-side rebalancing versus
> > external rebalancing.  That's why the KIP states that we will continue to
> > perform all the given partition rebalances immediately.  I was just
> > responding to the idea that maybe we should have an 

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

2019-06-19 Thread Stanislav Kozlovski
Hey there Colin,

Thanks for the work on this KIP. It is a much-needed improvement and I'm
excited to see it. Sorry for coming in so late to the discussion, I have
one question to better understand the change and a small suggestion.

I see we allow reassignment cancellation at the partition level - what is
the motivation behind that? I think that having the back-end structures
support it is a good choice since it allows us more flexibility in the
future but what are the reasons for allowing a user to cancel at a
partition level? I think allowing it might let users shoot themselves in
the foot easier and make tools harder to implement (needing to guard
against it).

In all regards, what do you think about an ease of use improvement where we
allow a user to cancel all reassignments for a topic without specifying its
partitions? Essentially, we could cancel all reassignments for a topic if
the Partitions field in AlterPartitionAssignmentsRequest is null.

Best,
Stanislav

On Mon, May 6, 2019 at 5:42 PM Colin McCabe  wrote:

> On Mon, May 6, 2019, at 07:39, Ismael Juma wrote:
> > Hi Colin,
> >
> > A quick comment.
> >
> > On Sat, May 4, 2019 at 11:18 PM Colin McCabe  wrote:
> >
> > > The big advantage of doing batching on the controller is that the
> > > controller has more information about what is going on in the
> cluster.  So
> > > it can schedule reassignments in a more optimal way.  For instance, it
> can
> > > schedule reassignments so that the load is distributed evenly across
> > > nodes.  This advantage is lost if we have to adhere to a rigid ordering
> > > that is set up in advance.  We don't know exactly when anything will
> > > complete in any case.  Just because one partition reassignment was
> started
> > > before another doesn't mean it will finish before another.
> >
> >
> > This is not quite true, right? The Controller doesn't know about
> partition
> > sizes, throughput per partition and other such information that external
> > tools like Cruise Control track.
>
> Hi Ismael,
>
> That's a good point, and one I should have included.
>
> I guess when I think about "do batching in the controller" versus "do
> batching in an external system" I tend to think about the information the
> controller could theoretically collect, rather than what it actually does
> :)  But certainly, adding this information to the controller would be a
> significant change, and maybe one we don't want to do if the external
> systems work well enough.
>
> Thinking about this a little bit more, I can see three advantages to
> controller-side batching.  Firstly, doing batching in the controller saves
> memory because we don't use a separate JVM, and don't duplicate the
> in-memory map of all the partitions.  Secondly, the information we're
> acting on would also be more up-to-date.  (I'm not sure how important this
> would be.)  Finally, it's one less thing to deploy.  I don't know if those
> are really enough to motivate switching now, but in a greenfield system I
> would probably choose controller-side rebalancing.
>
> In any case, this KIP is orthogonal to controller-side rebalancing versus
> external rebalancing.  That's why the KIP states that we will continue to
> perform all the given partition rebalances immediately.  I was just
> responding to the idea that maybe we should have an "ordering" of
> rebalancing partitions.  I don't think we want that, for controller-side
> rebalancing or externally batched rebalancing.
>
> best,
> Colin
>


-- 
Best,
Stanislav


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

2019-05-06 Thread Colin McCabe
On Mon, May 6, 2019, at 07:39, Ismael Juma wrote:
> Hi Colin,
> 
> A quick comment.
> 
> On Sat, May 4, 2019 at 11:18 PM Colin McCabe  wrote:
> 
> > The big advantage of doing batching on the controller is that the
> > controller has more information about what is going on in the cluster.  So
> > it can schedule reassignments in a more optimal way.  For instance, it can
> > schedule reassignments so that the load is distributed evenly across
> > nodes.  This advantage is lost if we have to adhere to a rigid ordering
> > that is set up in advance.  We don't know exactly when anything will
> > complete in any case.  Just because one partition reassignment was started
> > before another doesn't mean it will finish before another.
> 
> 
> This is not quite true, right? The Controller doesn't know about partition
> sizes, throughput per partition and other such information that external
> tools like Cruise Control track.

Hi Ismael,

That's a good point, and one I should have included.

I guess when I think about "do batching in the controller" versus "do batching 
in an external system" I tend to think about the information the controller 
could theoretically collect, rather than what it actually does :)  But 
certainly, adding this information to the controller would be a significant 
change, and maybe one we don't want to do if the external systems work well 
enough.

Thinking about this a little bit more, I can see three advantages to 
controller-side batching.  Firstly, doing batching in the controller saves 
memory because we don't use a separate JVM, and don't duplicate the in-memory 
map of all the partitions.  Secondly, the information we're acting on would 
also be more up-to-date.  (I'm not sure how important this would be.)  Finally, 
it's one less thing to deploy.  I don't know if those are really enough to 
motivate switching now, but in a greenfield system I would probably choose 
controller-side rebalancing.

In any case, this KIP is orthogonal to controller-side rebalancing versus 
external rebalancing.  That's why the KIP states that we will continue to 
perform all the given partition rebalances immediately.  I was just responding 
to the idea that maybe we should have an "ordering" of rebalancing partitions.  
I don't think we want that, for controller-side rebalancing or externally 
batched rebalancing.

best,
Colin


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

2019-05-06 Thread Ismael Juma
Hi Colin,

A quick comment.

On Sat, May 4, 2019 at 11:18 PM Colin McCabe  wrote:

> The big advantage of doing batching on the controller is that the
> controller has more information about what is going on in the cluster.  So
> it can schedule reassignments in a more optimal way.  For instance, it can
> schedule reassignments so that the load is distributed evenly across
> nodes.  This advantage is lost if we have to adhere to a rigid ordering
> that is set up in advance.  We don't know exactly when anything will
> complete in any case.  Just because one partition reassignment was started
> before another doesn't mean it will finish before another.


This is not quite true, right? The Controller doesn't know about partition
sizes, throughput per partition and other such information that external
tools like Cruise Control track.

Ismael


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

2019-05-06 Thread Viktor Somogyi-Vass
Hi Colin,

Thanks for explaining all this, it makes sense.

Viktor

On Sun, May 5, 2019 at 8:18 AM Colin McCabe  wrote:

> On Thu, May 2, 2019, at 09:35, Viktor Somogyi-Vass wrote:
> > Hey Colin & George,
> >
> > Thinking on George's points I was wondering if it's feasible to submit a
> > big reassignment to the controller and thus Zookeeper as frequent writes
> > are slow as the quorum has to synchronize. Perhaps it should be the
> > responsibility of KIP-435 
> but
> > I'd like to note it here as we're changing the current znode layout in
> this
> > KIP.
>
> Hi Viktor,
>
> This is similar conceptually to if we lose a broker from the cluster.  In
> that case, we have to remove that node from the ISR of all the partitions
> it has, which means updating O(partitions_on_node) znodes.  It's also
> similar to completing a reassignment in the existing Kafka version, and
> updating the partition znodes to reflect new nodes joining the ISR for
> various partitions.  While you are right that ZK is a low-bandwidth system,
> in general writing, to a few thousand ZNodes over the course of a second or
> two is OK.
>
> The existing reassignment znode requires the whole plan to fit within a
> single znode.  The maximum znodes size of 1 megabyte by default, and almost
> nobody reconfigures this.  Assuming about 100 bytes per reassignment, we
> can't get many more than about 10,000 partitions in a reassignment today in
> any case.  The current scalability bottleneck is much more on the side of
> "can kafka actually handle a huge amount of extra traffic due to ongoing
> reassignments"?
>
> That does bring up a good point, though-- we may want to have a "maximum
> concurrent reassignments" to avoid a common scenario that happens now,
> where people accidentally submit a plan that's way too big.  But this is
> not to protect ZooKeeper-- it is to protect the brokers.
>
> > I think ideally we should add these writes in batches to zookeeper and
> > otherwise store it in a replicated internal topic
> > (__partition_reassignments). That would solve the scalability problem as
> > the failover controller would be able to read it up very quickly and also
> > we would spread the writes in Zookeeper over time. Just the current,
> > actively replicated partitions should be present under
> > /brokers/topics/[topic]/partitions/[partitionId]/state, so those
> partitions
> > will know if they have to do reassignment (even in case of a broker
> > bounce). The controller on the other hand could regain its state by
> reading
> > up the last produced message from this __partition_reassignments topic
> and
> > reading up the Zookeeper state to figure out which batch its currently
> > doing (supposing it goes sequentially in the given reassignment).
>
> As I wrote in my reply to the other email, this is not needed because
> we're not adding any controller startup overhead beyond what already
> exists.  We do have some plans to optimize this, but it's outside the scope
> of this KIP.
>
> > I'll think a little bit more about this to fill out any gaps there are
> and
> > perhaps add it to my KIP. That being said probably we'll need to make
> some
> > benchmarking first if this bulk read-write causes a problem at all to
> avoid
> > premature optimisation. I generally don't really worry about reading up
> > this new information as the controller would read up the assignment
> anyway
> > in initializeControllerContext().
>
> Right, the controller will read those znodes on startup anyway.
>
> >
> > A question on SubmitPartitionReassignmentsRequest and its connection with
> > KIP-435 .
> Would
> > the list of topic-partitions have the same ordering on the client side as
> > well as the broker side? I think it would be an advantage as the user
> would
> > know in which order the reassignment would be performed. I think it's
> > useful when it comes to incrementalization as they'd be able to figure
> out
> > what replicas will be in one batch (given they know about the batch
> size).
>
> The big advantage of doing batching on the controller is that the
> controller has more information about what is going on in the cluster.  So
> it can schedule reassignments in a more optimal way.  For instance, it can
> schedule reassignments so that the load is distributed evenly across
> nodes.  This advantage is lost if we have to adhere to a rigid ordering
> that is set up in advance.  We don't know exactly when anything will
> complete in any case.  Just because one partition reassignment was started
> before another doesn't mean it will finish before another.
>
> Additionally, there may be multiple clients submitting assignments and
> multiple clients querying them.  So I don't think ordering makes sense here.
>
> best,
> Colin
>
> >
> > Viktor
> >
> > On Wed, May 1, 2019 at 8:33 AM George Li  .invalid>
> > wrote:
> >
> > >  Hi Colin,
> > >
> > > Thanks for 

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

2019-05-05 Thread Colin McCabe
On Thu, May 2, 2019, at 09:35, Viktor Somogyi-Vass wrote:
> Hey Colin & George,
> 
> Thinking on George's points I was wondering if it's feasible to submit a
> big reassignment to the controller and thus Zookeeper as frequent writes
> are slow as the quorum has to synchronize. Perhaps it should be the
> responsibility of KIP-435  but
> I'd like to note it here as we're changing the current znode layout in this
> KIP.

Hi Viktor,

This is similar conceptually to if we lose a broker from the cluster.  In that 
case, we have to remove that node from the ISR of all the partitions it has, 
which means updating O(partitions_on_node) znodes.  It's also similar to 
completing a reassignment in the existing Kafka version, and updating the 
partition znodes to reflect new nodes joining the ISR for various partitions.  
While you are right that ZK is a low-bandwidth system, in general writing, to a 
few thousand ZNodes over the course of a second or two is OK.

The existing reassignment znode requires the whole plan to fit within a single 
znode.  The maximum znodes size of 1 megabyte by default, and almost nobody 
reconfigures this.  Assuming about 100 bytes per reassignment, we can't get 
many more than about 10,000 partitions in a reassignment today in any case.  
The current scalability bottleneck is much more on the side of "can kafka 
actually handle a huge amount of extra traffic due to ongoing reassignments"?

That does bring up a good point, though-- we may want to have a "maximum 
concurrent reassignments" to avoid a common scenario that happens now, where 
people accidentally submit a plan that's way too big.  But this is not to 
protect ZooKeeper-- it is to protect the brokers.

> I think ideally we should add these writes in batches to zookeeper and
> otherwise store it in a replicated internal topic
> (__partition_reassignments). That would solve the scalability problem as
> the failover controller would be able to read it up very quickly and also
> we would spread the writes in Zookeeper over time. Just the current,
> actively replicated partitions should be present under
> /brokers/topics/[topic]/partitions/[partitionId]/state, so those partitions
> will know if they have to do reassignment (even in case of a broker
> bounce). The controller on the other hand could regain its state by reading
> up the last produced message from this __partition_reassignments topic and
> reading up the Zookeeper state to figure out which batch its currently
> doing (supposing it goes sequentially in the given reassignment).

As I wrote in my reply to the other email, this is not needed because we're not 
adding any controller startup overhead beyond what already exists.  We do have 
some plans to optimize this, but it's outside the scope of this KIP.

> I'll think a little bit more about this to fill out any gaps there are and
> perhaps add it to my KIP. That being said probably we'll need to make some
> benchmarking first if this bulk read-write causes a problem at all to avoid
> premature optimisation. I generally don't really worry about reading up
> this new information as the controller would read up the assignment anyway
> in initializeControllerContext().

Right, the controller will read those znodes on startup anyway.

> 
> A question on SubmitPartitionReassignmentsRequest and its connection with
> KIP-435 . Would
> the list of topic-partitions have the same ordering on the client side as
> well as the broker side? I think it would be an advantage as the user would
> know in which order the reassignment would be performed. I think it's
> useful when it comes to incrementalization as they'd be able to figure out
> what replicas will be in one batch (given they know about the batch size).

The big advantage of doing batching on the controller is that the controller 
has more information about what is going on in the cluster.  So it can schedule 
reassignments in a more optimal way.  For instance, it can schedule 
reassignments so that the load is distributed evenly across nodes.  This 
advantage is lost if we have to adhere to a rigid ordering that is set up in 
advance.  We don't know exactly when anything will complete in any case.  Just 
because one partition reassignment was started before another doesn't mean it 
will finish before another.

Additionally, there may be multiple clients submitting assignments and multiple 
clients querying them.  So I don't think ordering makes sense here.

best,
Colin

> 
> Viktor
> 
> On Wed, May 1, 2019 at 8:33 AM George Li 
> wrote:
> 
> >  Hi Colin,
> >
> > Thanks for KIP-455!  yes. KIP-236, etc. will depend on it.  It is the good
> > direction to go for the RP
> >
> > Regarding storing the new reassignments & original replicas at the
> > topic/partition level.  I have some concerns when controller is failing
> > over, and the scalability of scanning the active 

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

2019-05-03 Thread Colin McCabe
On Tue, Apr 30, 2019, at 23:33, George Li wrote:
>  Hi Colin,
> 
> Thanks for KIP-455!  yes. KIP-236, etc. will depend on it.  It is the 
> good direction to go for the RP 
> 
> Regarding storing the new reassignments & original replicas at the 
> topic/partition level.  I have some concerns when controller is failing 
> over, and the scalability of scanning the active reassignments from ZK 
> topic/partition level nodes. Please see my reply to Jason in the 
> KIP-236 thread. 

Hi George,

The controller already has to rescan this information from ZooKeeper when 
starting up, for unrelated reasons. 
 The controller needs to know about stuff like who is in the ISR for each 
partition, what the replicas are, and so forth.  So this doesn't add any 
additional overhead.

best,
Colin

> 
> Once the decision is made where new reassignment and original replicas 
> is stored, I will modify KIP-236 accordingly for how to cancel/rollback 
> the reassignments. 
> 
> Thanks,
> George 
> 
> 
> On Monday, April 15, 2019, 6:07:44 PM PDT, Colin McCabe 
>  wrote:  
>  
>  Hi all,
> 
> We've been having discussions on a few different KIPs (KIP-236, 
> KIP-435, etc.) about what the Admin Client replica reassignment API 
> should look like.  The current API is really hard to extend and 
> maintain, which is a big source of problems.  I think it makes sense to 
> have a KIP that establishes a clean API that we can use and extend 
> going forward, so I posted KIP-455.  Take a look.  :)
> 
> best,
> Colin
>


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

2019-05-02 Thread Viktor Somogyi-Vass
Hey Colin & George,

Thinking on George's points I was wondering if it's feasible to submit a
big reassignment to the controller and thus Zookeeper as frequent writes
are slow as the quorum has to synchronize. Perhaps it should be the
responsibility of KIP-435  but
I'd like to note it here as we're changing the current znode layout in this
KIP.
I think ideally we should add these writes in batches to zookeeper and
otherwise store it in a replicated internal topic
(__partition_reassignments). That would solve the scalability problem as
the failover controller would be able to read it up very quickly and also
we would spread the writes in Zookeeper over time. Just the current,
actively replicated partitions should be present under
/brokers/topics/[topic]/partitions/[partitionId]/state, so those partitions
will know if they have to do reassignment (even in case of a broker
bounce). The controller on the other hand could regain its state by reading
up the last produced message from this __partition_reassignments topic and
reading up the Zookeeper state to figure out which batch its currently
doing (supposing it goes sequentially in the given reassignment).
I'll think a little bit more about this to fill out any gaps there are and
perhaps add it to my KIP. That being said probably we'll need to make some
benchmarking first if this bulk read-write causes a problem at all to avoid
premature optimisation. I generally don't really worry about reading up
this new information as the controller would read up the assignment anyway
in initializeControllerContext().

A question on SubmitPartitionReassignmentsRequest and its connection with
KIP-435 . Would
the list of topic-partitions have the same ordering on the client side as
well as the broker side? I think it would be an advantage as the user would
know in which order the reassignment would be performed. I think it's
useful when it comes to incrementalization as they'd be able to figure out
what replicas will be in one batch (given they know about the batch size).

Viktor

On Wed, May 1, 2019 at 8:33 AM George Li 
wrote:

>  Hi Colin,
>
> Thanks for KIP-455!  yes. KIP-236, etc. will depend on it.  It is the good
> direction to go for the RPC.
>
> Regarding storing the new reassignments & original replicas at the
> topic/partition level.  I have some concerns when controller is failing
> over, and the scalability of scanning the active reassignments from ZK
> topic/partition level nodes. Please see my reply to Jason in the KIP-236
> thread.
>
> Once the decision is made where new reassignment and original replicas is
> stored, I will modify KIP-236 accordingly for how to cancel/rollback the
> reassignments.
>
> Thanks,
> George
>
>
> On Monday, April 15, 2019, 6:07:44 PM PDT, Colin McCabe <
> cmcc...@apache.org> wrote:
>
>  Hi all,
>
> We've been having discussions on a few different KIPs (KIP-236, KIP-435,
> etc.) about what the Admin Client replica reassignment API should look
> like.  The current API is really hard to extend and maintain, which is a
> big source of problems.  I think it makes sense to have a KIP that
> establishes a clean API that we can use and extend going forward, so I
> posted KIP-455.  Take a look.  :)
>
> best,
> Colin
>


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

2019-05-01 Thread George Li
 Hi Colin,

Thanks for KIP-455!  yes. KIP-236, etc. will depend on it.  It is the good 
direction to go for the RPC. 

Regarding storing the new reassignments & original replicas at the 
topic/partition level.  I have some concerns when controller is failing over, 
and the scalability of scanning the active reassignments from ZK 
topic/partition level nodes. Please see my reply to Jason in the KIP-236 
thread. 

Once the decision is made where new reassignment and original replicas is 
stored, I will modify KIP-236 accordingly for how to cancel/rollback the 
reassignments. 

Thanks,
George 


On Monday, April 15, 2019, 6:07:44 PM PDT, Colin McCabe 
 wrote:  
 
 Hi all,

We've been having discussions on a few different KIPs (KIP-236, KIP-435, etc.) 
about what the Admin Client replica reassignment API should look like.  The 
current API is really hard to extend and maintain, which is a big source of 
problems.  I think it makes sense to have a KIP that establishes a clean API 
that we can use and extend going forward, so I posted KIP-455.  Take a look.  :)

best,
Colin
  

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

2019-04-27 Thread Colin McCabe
Hi Viktor,

Good question.  The answer is that the Admin Client knows how to send messages 
to the controller node.  Take a look at electPreferredLeaders, createTopics, 
deleteTopics, etc. for some examples of how this is done in the code.

kafka-reassign-partitions.sh would be able to simply use the admin client for 
this functionality.  This would require the user to pass a --bootstrap-server 
argument, as you noted.  kafka-reassign-partitions.sh actually already has such 
an argument (this was added as part of the JBOD work...)

best,
Colin

On Fri, Apr 26, 2019, at 06:34, Viktor Somogyi-Vass wrote:
> Hi Colin,
> 
> How would the changed kafka-reassign-partitions.sh command work? Would it
> automatically figure out the controller node if you pass some bootstrap
> nodes with --bootstrap-server or are we deferring this implementation to
> the users?
> 
> Thanks,
> Viktor
> 
> On Sat, Apr 20, 2019 at 1:51 AM Colin McCabe  wrote:
> 
> > On Wed, Apr 17, 2019, at 17:23, Robert Barrett wrote:
> > > Thanks for the KIP, Colin. I have a couple questions:
> > >
> > > 1. What's the reasoning for requiring cancellation of a reassignment
> > before
> > > submitting a new one? It seems like overriding an existing reassignment
> > > could be done with a single update to
> > > /brokers/topics/[topic]/partitions/[partitionId]/state and a single
> > > LeaderAndIsrRequest. Maybe we could include a flag in the request so that
> > > the client can explicitly request to override an existing reassignment?
> >
> > Hmm, good point.  That might be more convenient than having to cancel and
> > remove before creating a new assignment.
> >
> > > 2. I agree that supporting the old ZK API for in the long term is a bad
> > > idea. However, while the number of tools that use the ZK API may be
> > small,
> > > this would be a non-trivial change for them. Could we temporarily support
> > > both, with a config enabling the new behavior to prevent users from
> > trying
> > > to use both mechanisms (if the config is true, the old znode is ignored;
> > if
> > > the config is false, the Admin Client API returns an error indicating
> > that
> > > it is not enabled)? We could then remove the ZK API in a later release,
> > to
> > > give people time to update their tools.
> >
> > It seems like the big change is basically just depending on adminclient
> > versus a ZK client.  The code itself for converting a JSON file into an
> > adminclient call shouldn't be difficult.  Maybe we could add a helper
> > method to do this, to make it easier to do the conversion.  We'll already
> > need that code for the command-line tool.
> >
> > best,
> > Colin
> >
> >
> > >
> > > Thanks,
> > > Bob
> > >
> > > On Mon, Apr 15, 2019 at 9:33 PM Colin McCabe  wrote:
> > >
> > > > link:
> > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > > >
> > > >
> > > >
> > > > On Mon, Apr 15, 2019, at 18:07, Colin McCabe wrote:
> > > > > Hi all,
> > > > >
> > > > > We've been having discussions on a few different KIPs (KIP-236,
> > > > > KIP-435, etc.) about what the Admin Client replica reassignment API
> > > > > should look like. The current API is really hard to extend and
> > > > > maintain, which is a big source of problems. I think it makes sense
> > to
> > > > > have a KIP that establishes a clean API that we can use and extend
> > > > > going forward, so I posted KIP-455. Take a look. :)
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > >
> >
>


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

2019-04-26 Thread Viktor Somogyi-Vass
Hi Colin,

How would the changed kafka-reassign-partitions.sh command work? Would it
automatically figure out the controller node if you pass some bootstrap
nodes with --bootstrap-server or are we deferring this implementation to
the users?

Thanks,
Viktor

On Sat, Apr 20, 2019 at 1:51 AM Colin McCabe  wrote:

> On Wed, Apr 17, 2019, at 17:23, Robert Barrett wrote:
> > Thanks for the KIP, Colin. I have a couple questions:
> >
> > 1. What's the reasoning for requiring cancellation of a reassignment
> before
> > submitting a new one? It seems like overriding an existing reassignment
> > could be done with a single update to
> > /brokers/topics/[topic]/partitions/[partitionId]/state and a single
> > LeaderAndIsrRequest. Maybe we could include a flag in the request so that
> > the client can explicitly request to override an existing reassignment?
>
> Hmm, good point.  That might be more convenient than having to cancel and
> remove before creating a new assignment.
>
> > 2. I agree that supporting the old ZK API for in the long term is a bad
> > idea. However, while the number of tools that use the ZK API may be
> small,
> > this would be a non-trivial change for them. Could we temporarily support
> > both, with a config enabling the new behavior to prevent users from
> trying
> > to use both mechanisms (if the config is true, the old znode is ignored;
> if
> > the config is false, the Admin Client API returns an error indicating
> that
> > it is not enabled)? We could then remove the ZK API in a later release,
> to
> > give people time to update their tools.
>
> It seems like the big change is basically just depending on adminclient
> versus a ZK client.  The code itself for converting a JSON file into an
> adminclient call shouldn't be difficult.  Maybe we could add a helper
> method to do this, to make it easier to do the conversion.  We'll already
> need that code for the command-line tool.
>
> best,
> Colin
>
>
> >
> > Thanks,
> > Bob
> >
> > On Mon, Apr 15, 2019 at 9:33 PM Colin McCabe  wrote:
> >
> > > link:
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> > >
> > >
> > >
> > > On Mon, Apr 15, 2019, at 18:07, Colin McCabe wrote:
> > > > Hi all,
> > > >
> > > > We've been having discussions on a few different KIPs (KIP-236,
> > > > KIP-435, etc.) about what the Admin Client replica reassignment API
> > > > should look like. The current API is really hard to extend and
> > > > maintain, which is a big source of problems. I think it makes sense
> to
> > > > have a KIP that establishes a clean API that we can use and extend
> > > > going forward, so I posted KIP-455. Take a look. :)
> > > >
> > > > best,
> > > > Colin
> > > >
> >
>


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

2019-04-19 Thread Colin McCabe
On Wed, Apr 17, 2019, at 17:23, Robert Barrett wrote:
> Thanks for the KIP, Colin. I have a couple questions:
> 
> 1. What's the reasoning for requiring cancellation of a reassignment before
> submitting a new one? It seems like overriding an existing reassignment
> could be done with a single update to
> /brokers/topics/[topic]/partitions/[partitionId]/state and a single
> LeaderAndIsrRequest. Maybe we could include a flag in the request so that
> the client can explicitly request to override an existing reassignment?

Hmm, good point.  That might be more convenient than having to cancel and 
remove before creating a new assignment.

> 2. I agree that supporting the old ZK API for in the long term is a bad
> idea. However, while the number of tools that use the ZK API may be small,
> this would be a non-trivial change for them. Could we temporarily support
> both, with a config enabling the new behavior to prevent users from trying
> to use both mechanisms (if the config is true, the old znode is ignored; if
> the config is false, the Admin Client API returns an error indicating that
> it is not enabled)? We could then remove the ZK API in a later release, to
> give people time to update their tools.

It seems like the big change is basically just depending on adminclient versus 
a ZK client.  The code itself for converting a JSON file into an adminclient 
call shouldn't be difficult.  Maybe we could add a helper method to do this, to 
make it easier to do the conversion.  We'll already need that code for the 
command-line tool.

best,
Colin


> 
> Thanks,
> Bob
> 
> On Mon, Apr 15, 2019 at 9:33 PM Colin McCabe  wrote:
> 
> > link:
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
> >
> > 
> >
> > On Mon, Apr 15, 2019, at 18:07, Colin McCabe wrote:
> > > Hi all,
> > >
> > > We've been having discussions on a few different KIPs (KIP-236,
> > > KIP-435, etc.) about what the Admin Client replica reassignment API
> > > should look like. The current API is really hard to extend and
> > > maintain, which is a big source of problems. I think it makes sense to
> > > have a KIP that establishes a clean API that we can use and extend
> > > going forward, so I posted KIP-455. Take a look. :)
> > >
> > > best,
> > > Colin
> > >
>


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

2019-04-17 Thread Robert Barrett
Thanks for the KIP, Colin. I have a couple questions:

1. What's the reasoning for requiring cancellation of a reassignment before
submitting a new one? It seems like overriding an existing reassignment
could be done with a single update to
/brokers/topics/[topic]/partitions/[partitionId]/state and a single
LeaderAndIsrRequest. Maybe we could include a flag in the request so that
the client can explicitly request to override an existing reassignment?
2. I agree that supporting the old ZK API for in the long term is a bad
idea. However, while the number of tools that use the ZK API may be small,
this would be a non-trivial change for them. Could we temporarily support
both, with a config enabling the new behavior to prevent users from trying
to use both mechanisms (if the config is true, the old znode is ignored; if
the config is false, the Admin Client API returns an error indicating that
it is not enabled)? We could then remove the ZK API in a later release, to
give people time to update their tools.

Thanks,
Bob

On Mon, Apr 15, 2019 at 9:33 PM Colin McCabe  wrote:

> link:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment
>
> C.
>
> On Mon, Apr 15, 2019, at 18:07, Colin McCabe wrote:
> > Hi all,
> >
> > We've been having discussions on a few different KIPs (KIP-236,
> > KIP-435, etc.) about what the Admin Client replica reassignment API
> > should look like. The current API is really hard to extend and
> > maintain, which is a big source of problems. I think it makes sense to
> > have a KIP that establishes a clean API that we can use and extend
> > going forward, so I posted KIP-455. Take a look. :)
> >
> > best,
> > Colin
> >


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

2019-04-15 Thread Colin McCabe
link: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment

C.

On Mon, Apr 15, 2019, at 18:07, Colin McCabe wrote:
> Hi all,
> 
> We've been having discussions on a few different KIPs (KIP-236, 
> KIP-435, etc.) about what the Admin Client replica reassignment API 
> should look like. The current API is really hard to extend and 
> maintain, which is a big source of problems. I think it makes sense to 
> have a KIP that establishes a clean API that we can use and extend 
> going forward, so I posted KIP-455. Take a look. :)
> 
> best,
> Colin
>