kafka-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From John Roesler <j...@confluent.io>
Subject Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams
Date Wed, 07 Aug 2019 19:13:38 GMT
Hey Guozhang,

Thanks for the review!

1. Yes, even with `num.standby.replicas := 0`, we will still temporarily
allocate standby tasks to accomplish a no-downtime task migration.
Although, I'd argue that this doesn't really violate the config, as the
task isn't a true hot standby. As soon as it catches up, we'll rebalance
again, that task will become active, and the original instance that hosted
the active task will no longer have the task assigned at all. Once the
stateDirCleaner kicks in, we'll free the disk space from it, and return to
the steady-state of having just one copy of the task in the cluster.

We can of course do without this, but I feel the current proposal is
operationally preferable, since it doesn't make configuring hot-standbys a
pre-requisite for fast rebalances.

2. Yes, I think your interpretation is what we intended. The default
balance_factor would be 1, as it is implicitly today. What this does is
allows operators to trade off less balanced assignments against fewer
rebalances. If you have lots of space capacity in your instances, this may
be a perfectly fine tradeoff, and you may prefer for Streams not to bother
streaming GBs of data from the broker in pursuit of perfect balance. Not
married to this configuration, though. It was inspired by the related work
research we did.

3. I'll take a look

3a. I think this is a good idea. I'd classify it as a type of grey failure
detection. It may make more sense to tackle grey failures as part of the
heartbeat protocol (as I POCed here:
https://github.com/apache/kafka/pull/7096/files). WDYT?

4. Good catch! I didn't think about that before. Looking at it now, though,
I wonder if we're actually protected already. The stateDirCleaner thread
only executes if the instance is in RUNNING state, and KIP-441 proposes to
use "probing rebalances" to report task lag. Hence, during the window
between when the instance reports a lag and the assignor makes a decision
about it, the instance should remain in REBALANCING state, right? If so,
then this should prevent the race condition. If not, then we do indeed need
to do something about it.

5. Good idea. I think that today, you can only see the consumer lag, which
is a poor substitute. I'll add some metrics to the proposal.

Thanks again for the comments!
-John

On Tue, Aug 6, 2019 at 4:27 PM Guozhang Wang <wangguoz@gmail.com> wrote:

> Hello Sophie,
>
> Thanks for the proposed KIP. I left some comments on the wiki itself, and I
> think I'm still not very clear on a couple or those:
>
> 1. With this proposal, does that mean with num.standby.replicas == 0, we
> may sometimes still have some standby tasks which may violate the config?
>
> 2. I think I understand the rationale to consider lags that is below the
> specified threshold to be equal, rather than still considering 5000 is
> better than 5001 -- we do not want to "over-optimize" and potentially falls
> into endless rebalances back and forth.
>
> But I'm not clear about the rationale of the second parameter of
> constrainedBalancedAssignment(StatefulTasksToRankedCandidates,
> balance_factor):
>
> Does that mean, e.g. with balance_factor of 3, we'd consider two
> assignments one resulting balance_factor 0 and one resulting balance_factor
> 3 to be equally optimized assignment and therefore may "stop early"? This
> was not very convincing to me :P
>
> 3. There are a couple of minor comments about the algorithm itself, left on
> the wiki page since it needs to refer to the exact line and better
> displayed there.
>
> 3.a Another wild thought about the threshold itself: today the assignment
> itself is memoryless, so we would not know if the reported `TaskLag` itself
> is increasing or decreasing even if the current value is under the
> threshold. I wonder if it worthy to make it a bit more complicated to track
> task lag trend at the assignor? Practically it may not be very uncommon
> that stand-by tasks are not keeping up due to the fact that other active
> tasks hosted on the same thread is starving the standby tasks.
>
> 4. There's a potential race condition risk when reporting `TaskLags` in the
> subscription: right after reporting it to the leader, the cleanup thread
> kicks in and deletes the state directory. If the task was assigned to the
> host it would cause it to restore from beginning and effectively make the
> seemingly optimized assignment very sub-optimal.
>
> To be on the safer side we should consider either prune out those tasks
> that are "close to be cleaned up" in the subscription, or we should delay
> the cleanup right after we've included them in the subscription in case
> they are been selected as assigned tasks by the assignor.
>
> 5. This is a meta comment: I think it would be helpful to add some user
> visibility on the standby tasks lagging as well, via metrics for example.
> Today it is hard for us to observe how far are our current standby tasks
> compared to the active tasks and whether that lag is being increasing or
> decreasing. As a follow-up task, for example, the rebalance should also be
> triggered if we realize that some standby task's lag is increasing
> indefinitely means that it cannot keep up (which is another indicator
> either you need to add more resources with the num.standbys or your are
> still not balanced enough).
>
>
> On Tue, Aug 6, 2019 at 1:32 PM Sophie Blee-Goldman <sophie@confluent.io>
> wrote:
>
> > Hey all,
> >
> > I'd like to kick off discussion on KIP-441, aimed at the long restore
> times
> > in Streams during which further active processing and IQ are blocked.
> > Please give it a read and let us know your thoughts
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-441:+Smooth+Scaling+Out+for+Kafka+Streams
> >
> > Cheers,
> > Sophie
> >
>
>
> --
> -- Guozhang
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message