Thanks for the message, Roger.

I think having 'acks=all' imply 'acks=minIsr' will probably result in some
improvement in the latency. However, I would note two things:

1. The numbers on the wiki are latency at max throughput, which should not
be representative of actual latency degradation. We should aim for more
representative numbers before trying to solve for the problem. I intend to
do this.
2. We have some toggles to reduce replication latency like
ReplicaFetchMinBytes, ReplicaFetchWaitMaxMs. We can improve the replication
latency by tuning these variables.
3. Engineers from Uber have played with the settings above and noticed that
our protocol is too inefficient to support frequent replica fetches. We
plan to improve this in the coming months.

If we solve the low hanging fruit in (3), we can probably have lower
latency configs in (2), which would mean that the actual impact of acks=all
reduces. In this context, acks=minIsr will probably show a smaller impact
on reducing replication latency. But it would definitely be interesting to
get those numbers and see whether the theory bears out in practice.

Regards,
Apurva

On Thu, Aug 31, 2017 at 8:56 AM, Roger Hoover <roger.hoo...@gmail.com>
wrote:

> Sorry, my math was sloppy.  It's not twice as many requests taking longer.
> If the probability of replication latency longer than X is Px for both
> replicas then,
>
> acks=all will have probability of Px(2-Px) of replication lag longer than X
> while
> acks=minIsr will be Px
>
>
> On Wed, Aug 30, 2017 at 5:18 PM, Roger Hoover <roger.hoo...@gmail.com>
> wrote:
>
> > Sorry if this is a bit out of left field but can't help wondering...
> >
> > One way to improve producer performance while still having good
> guarantees
> > would be to allow a setting between acks=1 and acks=all.  We could
> > introduce "acks=minIsr".  This is already the guarantee you get when the
> > ISR set shrinks below your replication factor.  Why not allow producers
> to
> > get notified when minIsr replication has been acheived even when the ISR
> > set is full?
> >
> > For rep factor == 3 and min.in.sync.replicas == 2 and sizeOf(ISR) == 3:
> > * with acks=all, the remote time of each request will be max(lag of 2
> > followers) whereas
> > * with acks=minIsr, the remote time of each request will be min(lag of 2
> > followers)
> >
> > Whatever your latency distribution is for replication, for any given
> > remote time (say 100 ms), twice as many requests take longer than that
> time
> > with acks=all vs acks=minIsr.
> >
> > Thoughts?
> >
> > Roger
> >
> > On Wed, Aug 30, 2017 at 4:52 PM, Apurva Mehta <apu...@confluent.io>
> wrote:
> >
> >> Hi Ted, int16 is sufficient. I forgot to specify initially. I have
> updated
> >> the KIP.
> >>
> >> Thanks for pointing it out!
> >> Apurva
> >>
> >> On Wed, Aug 30, 2017 at 4:43 PM, Ted Yu <yuzhih...@gmail.com> wrote:
> >>
> >> > For ProduceRequest v4, would int32 or int16 be enough for
> >> idempotenceLevel
> >> > ?
> >> >
> >> > Cheers
> >> >
> >> > On Wed, Aug 30, 2017 at 3:47 PM, Apurva Mehta <apu...@confluent.io>
> >> wrote:
> >> >
> >> > > Thanks Ismael and Jason, I filed a separate KIP to solve the
> problems
> >> > > identified through this discussion. I also incorporated Jason's
> >> comments
> >> > in
> >> > > that document:
> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > 192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled
> >> > >
> >> > > Please have a look,
> >> > > Apurva
> >> > >
> >> > > On Tue, Aug 29, 2017 at 3:28 AM, Ismael Juma <ism...@juma.me.uk>
> >> wrote:
> >> > >
> >> > > > Thanks for the proposals. I think they make sense and I also agree
> >> with
> >> > > > Jason's suggestions. Also, it would be good to include the updated
> >> > > > ProduceRequest/Response schema in the KIP.
> >> > > >
> >> > > > Ismael
> >> > > >
> >> > > > On Tue, Aug 22, 2017 at 11:42 PM, Jason Gustafson <
> >> ja...@confluent.io>
> >> > > > wrote:
> >> > > >
> >> > > > > Thanks Apurva,
> >> > > > >
> >> > > > > On compatibility: I think the proposal makes sense. It's a pity
> >> that
> >> > we
> >> > > > > can't support idempotence for 0.11.0.0 brokers in the "safe"
> mode
> >> > even
> >> > > if
> >> > > > > it is supported by the broker. I can already imagine users
> >> > complaining
> >> > > > > about this, but I guess it's the consequence of missing the
> >> impact of
> >> > > > that
> >> > > > > validation check and not thinking through the ultimate goal of
> >> > enabling
> >> > > > > idempotence by default. A couple minor comments:
> >> > > > >
> >> > > > > 1. Instead of "safe," Ismael suggested "requested" as an
> >> alternative.
> >> > > > That
> >> > > > > seems to suggest more clearly that idempotence will only be used
> >> when
> >> > > the
> >> > > > > broker supports it.
> >> > > > > 2. Should we deprecate the "true" and "false" options? It's a
> >> little
> >> > > > weird
> >> > > > > long term to support them in addition to the descriptive names.
> >> > > > >
> >> > > > > On the OutOfOrderSequence proposal: high-level, the design makes
> >> > > sense. A
> >> > > > > couple questions:
> >> > > > >
> >> > > > > 1. With this proposal, OutOfOrderSequence means that we must
> have
> >> a
> >> > > last
> >> > > > > produced offset. Is the idea to expose that in the
> >> > > > > OutOfOrderSequenceException so that users know which data was
> >> lost?
> >> > > > > 2. Previously we discussed duplicate handling. Currently we
> raise
> >> > > > > OutOfOrderSequence if we happen to get a sequence number which
> is
> >> > > earlier
> >> > > > > than the sequence numbers we have cached. Alternatively, you
> >> > suggested
> >> > > we
> >> > > > > can return a separate DuplicateError for this case, which
> clients
> >> can
> >> > > > > ignore if they do not care about the offset. I think it might
> make
> >> > > sense
> >> > > > to
> >> > > > > include that here so that the OutOfOrderSequence error is
> >> > unambiguous.
> >> > > > >
> >> > > > > Finally, do you plan to roll these proposals into the current
> KIP
> >> or
> >> > do
> >> > > > > them separately? Probably makes sense to combine them since they
> >> both
> >> > > > > require a bump to the ProduceRequest.
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Jason
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > On Fri, Aug 18, 2017 at 5:18 PM, Apurva Mehta <
> >> apu...@confluent.io>
> >> > > > wrote:
> >> > > > >
> >> > > > > > Thanks Jason and Ismael.
> >> > > > > >
> >> > > > > > The message format problem is an acute one: if we enable
> >> > idempotence
> >> > > by
> >> > > > > > default, the UnsupportedVersionException when writing to
> topics
> >> > with
> >> > > > the
> >> > > > > > older message format would mean that our prescribed upgrade
> >> steps
> >> > > would
> >> > > > > not
> >> > > > > > work. I have detailed the problems and the solutions on this
> >> page
> >> > > > (linked
> >> > > > > > to from the wiki):
> >> > > > > >
> >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> >> > > > > > Kafka+Exactly+Once+-+Dealing+with+older+message+formats+
> >> > > > > > when+idempotence+is+enabled
> >> > > > > >
> >> > > > > > It is worth discussing the solution to the problem proposed
> >> there.
> >> > If
> >> > > > it
> >> > > > > is
> >> > > > > > conceptually sound, it doesnt' seem too hard to implement.
> >> > > > > >
> >> > > > > > As far as the other problem of the spurious OutOfOrderSequence
> >> > > > problem, I
> >> > > > > > have documented a proposed solution here:
> >> > > > > >
> >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> >> > > > > > Kafka+Exactly+Once+-+Solving+the+problem+of+spurious+
> >> > > > > > OutOfOrderSequence+errors
> >> > > > > >
> >> > > > > > This solution is a bit more involved in terms of effort.
> >> > > > > >
> >> > > > > > I think we cannot make the idempotent producer the default
> >> unless
> >> > we
> >> > > > > solve
> >> > > > > > the message format compatibility problem. I would also prefer
> to
> >> > > solve
> >> > > > > the
> >> > > > > > second problem before making idempotence the default.
> >> > > > > >
> >> > > > > > I would be interested to hear everyone's thoughts on the two
> >> > > solutions
> >> > > > > > proposed above.
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Apurva
> >> > > > > >
> >> > > > > > On Fri, Aug 18, 2017 at 9:24 AM, Jason Gustafson <
> >> > ja...@confluent.io
> >> > > >
> >> > > > > > wrote:
> >> > > > > >
> >> > > > > > > >
> >> > > > > > > >  so this change will break client backward compatibility
> >> while
> >> > > > > > connecting
> >> > > > > > > > to 0.10.X brokers.
> >> > > > > > > >  users need to change producer default settings while
> >> > connecting
> >> > > > > older
> >> > > > > > > > brokers.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > At the moment, I think the answer is yes. The old broker
> will
> >> not
> >> > > > > support
> >> > > > > > > the InitProducerId request, so the producer will immediately
> >> > fail.
> >> > > > > > Similar
> >> > > > > > > to the handling of old message formats mentioned above, we
> >> > probably
> >> > > > > need
> >> > > > > > to
> >> > > > > > > change this so that we just revert to old producer semantics
> >> if
> >> > the
> >> > > > > > broker
> >> > > > > > > can't support idempotence.
> >> > > > > > >
> >> > > > > > > -Jason
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Fri, Aug 18, 2017 at 8:48 AM, Manikumar <
> >> > > > manikumar.re...@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > 3. The message format requirement is a good point. This
> >> > should
> >> > > be
> >> > > > > > > > mentioned
> >> > > > > > > > > in the compatibility section. Users who are still using
> >> the
> >> > old
> >> > > > > > message
> >> > > > > > > > > format will get an error after the upgrade, right?
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > > >  so this change will break client backward compatibility
> >> while
> >> > > > > > connecting
> >> > > > > > > > to 0.10.X brokers.
> >> > > > > > > >  users need to change producer default settings while
> >> > connecting
> >> > > > > older
> >> > > > > > > > brokers.
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Reply via email to