Hi, Jason,
Thanks for the comment. I didn't realize that the KIP was only focusing on
the latency of the producer. From an application's perspective, if an
application is latency sensitive, it probably wants to optimize the
end-to-end latency from the producer to the consumer. Litao, would you
agr
Initially I thought this proposal was just about improving producer
latency. So acks=quorum (or whatever) was sort of the midway point between
acks=1 and acks=all, but offsets would only be exposed according to current
high watermark semantics (meaning full ISR replication). However, it looks
like
Hi, Colin,
A couple of thoughts on your previous comments.
1. I am not sure if the controller is necessarily a bottleneck for leader
election. In the common case, leader election is due to a single broker
failure. So, the controller only needs to change the leaders for the number
of partitions on
Hi Litao,
When acks=all is used, we will only require all ISR replicas to ack before
returning, your understanding on this part is right.
What I meant is, that with acks=all, for the above example with {A, B, C},
if the ISR list still contains all three replicas, we will still require
ALL 3 of th
Hey Guozhang. Not fully understand your following comments.
The goal of my approach is to maintain the behavior of ack="all", which
> happen to do better than what Kafka is actually guaranteed: when both A and
> B are partitioned off, produced records will not be acked since "all"
> requires all r
Folks. Thanks for all of the good discussions.
Here are a few of my thoughts:
1. we won't change any existing semantics. That means, besides acks '-1
(all)', '0', '1', we will introduce a separate 'quorum' and document the
semantic. 'quorum' is a totally different view of our replication
Hello Tao,
No I was not proposing to change the mechanism of acks=all, and only
mentioning that today even with acks=all the tolerance of failures is
theoretically still bounded by min.isr settings though we do require all
replicas in ISR (which may be larger than min.isr) to replicate before
resp
Hey Colin,
Yeah I think you have a good point that controller is currently a
scalability bottleneck in Kafka and it will be great if we can re-design it
in distributed way using some consensus protocol. I have not gone through
the available options here but I imagine this will require a lot of
dis
On Wed, Jan 24, 2018, at 15:59, Jun Rao wrote:
> Hi, Litao,
>
> Thanks for the KIP. Good proposal. A few comments below.
>
> 1. The KIP says "select the live replica with the largest LEO". I guess
> what you meant is selecting the live replicas in ISR with the largest LEO?
>
> 2. I agree that w
Hi Dong & Guozhang,
Thanks... this is a really interesting discussion!
My first thought is, why should the controller be involved in choosing the
leader of the quorum? That seems like an internal quorum issue. All of the
state that is needed to select the proper leader is on the nodes themsel
Hey Guozhang,
I don't have very detailed design. But I have some high level idea that can
probably work. Here is how it looks like:
- When controller needs to do leadership movement
- Controller sends a request asking the brokers involved in the
migration to use acks=[isr_set] and wait for res
Hi Guozhang,
Are you proposing changing semantic of ack=all to acknowledge message only
after all replicas (not all ISRs, which is what Kafka currently is doing)
have committed the message? This is equivalent to setting min.isr=number of
replicas, which makes ack=all much stricter than what Kafka
Hi Dong,
Could you elaborate a bit more how controller could affect leaders to
switch between all and quorum?
Guozhang
On Fri, Feb 2, 2018 at 10:12 PM, Dong Lin wrote:
> Hey Guazhang,
>
> Got it. Thanks for the detailed explanation. I guess my point is that we
> can probably achieve the best
Hey Guazhang,
Got it. Thanks for the detailed explanation. I guess my point is that we
can probably achieve the best of both worlds, i.e. maintain the existing
behavior of ack="all" while improving the tail latency.
Thanks,
Dong
On Fri, Feb 2, 2018 at 8:43 PM, Guozhang Wang wrote:
> Hi Dong,
Hi Dong,
Yes, in terms of fault tolerance "quorum" does not do better than "all", as
I said, with {min.isr} to X+1 Kafka is able to tolerate X failures only. So
if A and B are partitioned off at the same time, then there are two
concurrent failures and we do not guarantee all acked messages will b
Hey Guozhang,
According to the new proposal, with 3 replicas, min.isr=2 and
acks="quorum", it seems that acknowledged messages can still be truncated
in the network partition scenario you mentioned, right? So I guess the goal
is for some user to achieve better tail latency at the cost of potential
Hello Litao,
Just double checking on the leader election details, do you have time to
complete the proposal on that part?
Also Jun mentioned one caveat related to KIP-250 on the KIP-232 discussion
thread that Dong is working on, I figured it is worth pointing out here
with a tentative solution:
Hey Litao,
Not sure there will be an easy way to select the broker with highest LEO
without losing acknowledged message. In case it is useful, here is another
idea. Maybe we can have a mechanism to turn switch between the min.isr and
isr set for determining when to acknowledge a message. Controlle
Thanks Jun for the detailed feedback.
Yes, for #1, I mean the live replicas from the ISR.
Actually, I believe for all of the 4 new leader election strategies
(offline, reassign, preferred replica and controlled shutdown), we need to
make corresponding changes. Will document the details in the KIP
Thanks for the KIP Litao.
1. I agree with Dong that it would be better to reuse on the existing
config if possible, and with that regards I also agree with Jun's point #2
that previously, there is a confusion on the min.isr's semantics from the
user's perspective, and we learned that it is actuall
Hi, Litao,
Thanks for the KIP. Good proposal. A few comments below.
1. The KIP says "select the live replica with the largest LEO". I guess
what you meant is selecting the live replicas in ISR with the largest LEO?
2. I agree that we can probably just reuse the current min.isr
configuration, bu
Sorry folks, just realized I didn't use the correct thread format for the
discussion. I started this new one and copied all of the responses from the
old one.
@Dong
It makes sense to just use the min.insync.replicas instead of introducing a
new config, and we must make this change together with th
Hey folks. I would like to add a feature to support the quorum-based
acknowledgment for the producer request. We have been running a modified
version of Kafka on our testing cluster for weeks, the improvement of P999
is significant with very stable latency. Additionally, I have a proposal to
achiev
Hey folks. I would like to add a feature to support the quorum-based
acknowledgment for the producer request. We have been running a
modified version of Kafka on our testing cluster for weeks, the
improvement of P999 is significant with very stable latency.
Additionally, I have a proposal to achiev
24 matches
Mail list logo