Hi Dong,

Thanks for your valuable comments. Please see my reply below.

1. The Google doc showed only 1 partition. Now let's consider a more common
scenario
where broker0 is the leader of many partitions. And let's say for some
reason its IO becomes slow.
The number of leader partitions on broker0 is so large, say 10K, that the
cluster is skewed,
and the operator would like to shift the leadership for a lot of
partitions, say 9K, to other brokers,
either manually or through some service like cruise control.
With this KIP, not only will the leadership transitions finish more
quickly, helping the cluster itself becoming more balanced,
but all existing producers corresponding to the 9K partitions will get the
errors relatively quickly
rather than relying on their timeout, thanks to the batched async ZK
operations.
To me it's a useful feature to have during such troublesome times.


2. The experiments in the Google Doc have shown that with this KIP many
producers
receive an explicit error NotLeaderForPartition, based on which they retry
immediately.
Therefore the latency (~14 seconds+quick retry) for their single message is
much smaller
compared with the case of timing out without the KIP (30 seconds for timing
out + quick retry).
One might argue that reducing the timing out on the producer side can
achieve the same result,
yet reducing the timeout has its own drawbacks[1].

Also *IF* there were a metric to show the number of truncated messages on
brokers,
with the experiments done in the Google Doc, it should be easy to see that
a lot fewer messages need
to be truncated on broker0 since the up-to-date metadata avoids appending
of messages
in subsequent PRODUCE requests. If we talk to a system operator and ask
whether
they prefer fewer wasteful IOs, I bet most likely the answer is yes.

3. To answer your question, I think it might be helpful to construct some
formulas.
To simplify the modeling, I'm going back to the case where there is only
ONE partition involved.
Following the experiments in the Google Doc, let's say broker0 becomes the
follower at time t0,
and after t0 there were still N produce requests in its request queue.
With the up-to-date metadata brought by this KIP, broker0 can reply with an
NotLeaderForPartition exception,
let's use M1 to denote the average processing time of replying with such an
error message.
Without this KIP, the broker will need to append messages to segments,
which may trigger a flush to disk,
let's use M2 to denote the average processing time for such logic.
Then the average extra latency incurred without this KIP is N * (M2 - M1) /
2.

In practice, M2 should always be larger than M1, which means as long as N
is positive,
we would see improvements on the average latency.
There does not need to be significant backlog of requests in the request
queue,
or severe degradation of disk performance to have the improvement.

Regards,
Lucas


[1] For instance, reducing the timeout on the producer side can trigger
unnecessary duplicate requests
when the corresponding leader broker is overloaded, exacerbating the
situation.

On Sun, Jul 1, 2018 at 9:18 PM, Dong Lin <lindon...@gmail.com> wrote:

> Hey Lucas,
>
> Thanks much for the detailed documentation of the experiment.
>
> Initially I also think having a separate queue for controller requests is
> useful because, as you mentioned in the summary section of the Google doc,
> controller requests are generally more important than data requests and we
> probably want controller requests to be processed sooner. But then Eno has
> two very good questions which I am not sure the Google doc has answered
> explicitly. Could you help with the following questions?
>
> 1) It is not very clear what is the actual benefit of KIP-291 to users. The
> experiment setup in the Google doc simulates the scenario that broker is
> very slow handling ProduceRequest due to e.g. slow disk. It currently
> assumes that there is only 1 partition. But in the common scenario, it is
> probably reasonable to assume that there are many other partitions that are
> also actively produced to and ProduceRequest to these partition also takes
> e.g. 2 seconds to be processed. So even if broker0 can become follower for
> the partition 0 soon, it probably still needs to process the ProduceRequest
> slowly t in the queue because these ProduceRequests cover other partitions.
> Thus most ProduceRequest will still timeout after 30 seconds and most
> clients will still likely timeout after 30 seconds. Then it is not
> obviously what is the benefit to client since client will timeout after 30
> seconds before possibly re-connecting to broker1, with or without KIP-291.
> Did I miss something here?
>
> 2) I guess Eno's is asking for the specific benefits of this KIP to user or
> system administrator, e.g. whether this KIP decreases average latency,
> 999th percentile latency, probably of exception exposed to client etc. It
> is probably useful to clarify this.
>
> 3) Does this KIP help improve user experience only when there is issue with
> broker, e.g. significant backlog in the request queue due to slow disk as
> described in the Google doc? Or is this KIP also useful when there is no
> ongoing issue in the cluster? It might be helpful to clarify this to
> understand the benefit of this KIP.
>
>
> Thanks much,
> Dong
>
>
>
>
> On Fri, Jun 29, 2018 at 2:58 PM, Lucas Wang <lucasatu...@gmail.com> wrote:
>
> > Hi Eno,
> >
> > Sorry for the delay in getting the experiment results.
> > Here is a link to the positive impact achieved by implementing the
> proposed
> > change:
> > https://docs.google.com/document/d/1ge2jjp5aPTBber6zaIT9AdhW
> > FWUENJ3JO6Zyu4f9tgQ/edit?usp=sharing
> > Please take a look when you have time and let me know your feedback.
> >
> > Regards,
> > Lucas
> >
> > On Tue, Jun 26, 2018 at 9:52 AM, Harsha <ka...@harsha.io> wrote:
> >
> > > Thanks for the pointer. Will take a look might suit our requirements
> > > better.
> > >
> > > Thanks,
> > > Harsha
> > >
> > > On Mon, Jun 25th, 2018 at 2:52 PM, Lucas Wang <lucasatu...@gmail.com>
> > > wrote:
> > >
> > > >
> > > >
> > > >
> > > > Hi Harsha,
> > > >
> > > > If I understand correctly, the replication quota mechanism proposed
> in
> > > > KIP-73 can be helpful in that scenario.
> > > > Have you tried it out?
> > > >
> > > > Thanks,
> > > > Lucas
> > > >
> > > >
> > > >
> > > > On Sun, Jun 24, 2018 at 8:28 AM, Harsha < ka...@harsha.io > wrote:
> > > >
> > > > > Hi Lucas,
> > > > > One more question, any thoughts on making this configurable
> > > > > and also allowing subset of data requests to be prioritized. For
> > > example
> > > >
> > > > > ,we notice in our cluster when we take out a broker and bring new
> one
> > > it
> > > >
> > > > > will try to become follower and have lot of fetch requests to other
> > > > leaders
> > > > > in clusters. This will negatively effect the application/client
> > > > requests.
> > > > > We are also exploring the similar solution to de-prioritize if a
> new
> > > > > replica comes in for fetch requests, we are ok with the replica to
> be
> > > > > taking time but the leaders should prioritize the client requests.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Harsha
> > > > >
> > > > > On Fri, Jun 22nd, 2018 at 11:35 AM Lucas Wang wrote:
> > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Hi Eno,
> > > > > >
> > > > > > Sorry for the delayed response.
> > > > > > - I haven't implemented the feature yet, so no experimental
> results
> > > so
> > > >
> > > > > > far.
> > > > > > And I plan to test in out in the following days.
> > > > > >
> > > > > > - You are absolutely right that the priority queue does not
> > > completely
> > > >
> > > > > > prevent
> > > > > > data requests being processed ahead of controller requests.
> > > > > > That being said, I expect it to greatly mitigate the effect of
> > stable
> > > > > > metadata.
> > > > > > In any case, I'll try it out and post the results when I have it.
> > > > > >
> > > > > > Regards,
> > > > > > Lucas
> > > > > >
> > > > > > On Wed, Jun 20, 2018 at 5:44 AM, Eno Thereska <
> > > eno.there...@gmail.com
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Lucas,
> > > > > > >
> > > > > > > Sorry for the delay, just had a look at this. A couple of
> > > questions:
> > > >
> > > > > > > - did you notice any positive change after implementing this
> KIP?
> > > > I'm
> > > > > > > wondering if you have any experimental results that show the
> > > benefit
> > > > of
> > > > > > the
> > > > > > > two queues.
> > > > > > >
> > > > > > > - priority is usually not sufficient in addressing the problem
> > the
> > > > KIP
> > > > > > > identifies. Even with priority queues, you will sometimes
> > (often?)
> > > > have
> > > > > > the
> > > > > > > case that data plane requests will be ahead of the control
> plane
> > > > > > requests.
> > > > > > > This happens because the system might have already started
> > > > processing
> > > > > > the
> > > > > > > data plane requests before the control plane ones arrived. So
> it
> > > > would
> > > > > > be
> > > > > > > good to know what % of the problem this KIP addresses.
> > > > > > >
> > > > > > > Thanks
> > > > > > > Eno
> > > > > > >
> > > > > >
> > > > > >
> > > > > > > On Fri, Jun 15, 2018 at 4:44 PM, Ted Yu < yuzhih...@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Change looks good.
> > > > > > > >
> > > > > > > > Thanks
> > > > > > > >
> > > > > > > > On Fri, Jun 15, 2018 at 8:42 AM, Lucas Wang <
> > > lucasatu...@gmail.com
> > > >
> > > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Ted,
> > > > > > > > >
> > > > > > > > > Thanks for the suggestion. I've updated the KIP. Please
> take
> > > > > another
> > > > > >
> > > > > > > > look.
> > > > > > > > >
> > > > > > > > > Lucas
> > > > > > > > >
> > > > > > > > > On Thu, Jun 14, 2018 at 6:34 PM, Ted Yu <
> > yuzhih...@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Currently in KafkaConfig.scala :
> > > > > > > > > >
> > > > > > > > > > val QueuedMaxRequests = 500
> > > > > > > > > >
> > > > > > > > > > It would be good if you can include the default value for
> > > this
> > > >
> > > > > new
> > > > > >
> > > > > > > > config
> > > > > > > > > > in the KIP.
> > > > > > > > > >
> > > > > > > > > > Thanks
> > > > > > > > > >
> > > > > > > > > > On Thu, Jun 14, 2018 at 4:28 PM, Lucas Wang <
> > > > > lucasatu...@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Ted, Dong
> > > > > > > > > > >
> > > > > > > > > > > I've updated the KIP by adding a new config, instead of
> > > > reusing
> > > > > > the
> > > > > > > > > > > existing one.
> > > > > > > > > > > Please take another look when you have time. Thanks a
> > lot!
> > > > > > > > > > >
> > > > > > > > > > > Lucas
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Jun 14, 2018 at 2:33 PM, Ted Yu <
> > > yuzhih...@gmail.com
> > > >
> > > > > >
> > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > bq. that's a waste of resource if control request
> rate
> > is
> > > > low
> > > > > > > > > > > >
> > > > > > > > > > > > I don't know if control request rate can get to
> > 100,000,
> > > > > > likely
> > > > > > > > not.
> > > > > > > > > > Then
> > > > > > > > > > > > using the same bound as that for data requests seems
> > > high.
> > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Jun 13, 2018 at 10:13 PM, Lucas Wang <
> > > > > > > > lucasatu...@gmail.com >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Ted,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for taking a look at this KIP.
> > > > > > > > > > > > > Let's say today the setting of
> "queued.max.requests"
> > in
> > > > > > > cluster A
> > > > > > > > > is
> > > > > > > > > > > > 1000,
> > > > > > > > > > > > > while the setting in cluster B is 100,000.
> > > > > > > > > > > > > The 100 times difference might have indicated that
> > > > machines
> > > > > > in
> > > > > > > > > > cluster
> > > > > > > > > > > B
> > > > > > > > > > > > > have larger memory.
> > > > > > > > > > > > >
> > > > > > > > > > > > > By reusing the "queued.max.requests", the
> > > > > > controlRequestQueue
> > > > > > > in
> > > > > > > > > > > cluster
> > > > > > > > > > > > B
> > > > > > > > > > > > > automatically
> > > > > > > > > > > > > gets a 100x capacity without explicitly bothering
> the
> > > > > > > operators.
> > > > > > > > > > > > > I understand the counter argument can be that maybe
> > > > that's
> > > > > a
> > > > > >
> > > > > > > > waste
> > > > > > > > > of
> > > > > > > > > > > > > resource if control request
> > > > > > > > > > > > > rate is low and operators may want to fine tune the
> > > > > capacity
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > > > controlRequestQueue.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I'm ok with either approach, and can change it if
> you
> > > or
> > > >
> > > > > > anyone
> > > > > > > > > else
> > > > > > > > > > > > feels
> > > > > > > > > > > > > strong about adding the extra config.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Lucas
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Jun 13, 2018 at 3:11 PM, Ted Yu <
> > > > > yuzhih...@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Lucas:
> > > > > > > > > > > > > > Under Rejected Alternatives, #2, can you
> elaborate
> > a
> > > > bit
> > > > > > more
> > > > > > > > on
> > > > > > > > > > why
> > > > > > > > > > > > the
> > > > > > > > > > > > > > separate config has bigger impact ?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Jun 13, 2018 at 2:00 PM, Dong Lin <
> > > > > > > lindon...@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hey Luca,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the KIP. Looks good overall. Some
> > > > comments
> > > > > > > below:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - We usually specify the full mbean for the new
> > > > metrics
> > > > > > in
> > > > > > > > the
> > > > > > > > > > KIP.
> > > > > > > > > > > > Can
> > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > specify it in the Public Interface section
> > similar
> > > > to
> > > > > > > KIP-237
> > > > > > > > > > > > > > > < https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > >
> > > > > > > > > > > > > > > 237%3A+More+Controller+Health+Metrics>
> > > > > > > > > > > > > > > ?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Maybe we could follow the same pattern as
> > KIP-153
> > > > > > > > > > > > > > > < https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > >
> > > > > > > > > > > > > > >
> > > > 153%3A+Include+only+client+traffic+in+BytesOutPerSec+
> > > > > > > > metric>,
> > > > > > > > > > > > > > > where we keep the existing sensor name
> > > > "BytesInPerSec"
> > > > > > and
> > > > > > > > add
> > > > > > > > > a
> > > > > > > > > > > new
> > > > > > > > > > > > > > sensor
> > > > > > > > > > > > > > > "ReplicationBytesInPerSec", rather than
> replacing
> > > > the
> > > > > > > sensor
> > > > > > > > > > name "
> > > > > > > > > > > > > > > BytesInPerSec" with e.g. "ClientBytesInPerSec".
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - It seems that the KIP changes the semantics
> of
> > > the
> > > >
> > > > > > broker
> > > > > > > > > > config
> > > > > > > > > > > > > > > "queued.max.requests" because the number of
> total
> > > > > > requests
> > > > > > > > > queued
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > broker will be no longer bounded by
> > > > > > "queued.max.requests".
> > > > > > > > This
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > needs to be specified in the Public Interfaces
> > > > section
> > > > > > for
> > > > > > > > > > > > discussion.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Jun 13, 2018 at 12:45 PM, Lucas Wang <
> > > > > > > > > > > lucasatu...@gmail.com >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Kafka experts,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I created KIP-291 to add a separate queue for
> > > > > > controller
> > > > > > > > > > > requests:
> > > > > > > > > > > > > > > > https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > >
> > > > > > > 291%
> > > > > > > > > > > > > > > > 3A+Have+separate+queues+for+
> > > > > control+requests+and+data+
> > > > > >
> > > > > > > > > requests
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Can you please take a look and let me know
> your
> > > > > > feedback?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks a lot for your time!
> > > > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > > > Lucas
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > >
> >
>

Reply via email to