Not if the system does ordered delivery and the id is a sequentially
increasing integer (mod something).

Essentially you need to keep
  (producer_id, topic, partition, max_seq_num)
on each broker. That is, rather than storing all keys you just need to know
the max you have seen if the seq_num is < max_seq_num+1 you have a
duplicate delivery. You can think of this as a persistent form of TCP's
sequence numbers, but instead of lasting only with the transient connection
it is a permanent part of the topic which is replicated with the message.

There are still a number of questions: How do producers get their ids? How
does the client initialize its sequence number? How do you keep the number
of producer ids bounded? etc.

If we did this it would be a ways out still. It would likely be an opt-in
client feature...i.e. clients that don't support it will work as normal. I
will try to get a more detailed design up in the next few weeks just so
there is something more concrete that tries to work out all the issues.

-Jay




On Wed, Aug 7, 2013 at 5:50 PM, Milind Parikh <milindpar...@gmail.com>wrote:

> Interesting. .. wouldn't the producer sequence grow without bounds, in the
> first case, even with the simpler non-ha of key assumption, to provide a
> strict exactly once semantics?
>
> In other words, wouldn't you need to store the entire set of keys that the
> broker has ever seen to ensure that a potential replayed message doesn't
> make it into the commit; given multiple producers?
>
> In mps (github.com/milindparikh/mps), I use a rotating double bloom filter
> to provide a "nearly exactly once" semantics to prevent an without-bound
> growth of such a sequence.
>
> Regards
> Milind
> On Aug 7, 2013 4:26 PM, "Jay Kreps" <jay.kr...@gmail.com> wrote:
>
> > Yeah I'm not sure how good our understanding was when we wrote that.
> >
> > Here is my take now:
> >
> > At least once delivery is not that hard but you need the ability to
> > deduplicate things--basically you turn the "at least once delivery
> channel"
> > into the "exactly once channel" by throwing away duplicates. This means
> > 1. Some key assigned by the producer that allows the broker to detect a
> > re-published message to make publishing idempotent. This solves the
> problem
> > of producer retries. This key obviously has to be highly available--i.e.
> if
> > the leader for a partition fails the follower must correctly deduplicate
> > for all committed messages.
> > 2. Some key that allows the consumer to detect a re-consumed message.
> >
> > The first item is actually pretty doable as we can track some producer
> > sequence in the log and use it to avoid duplicate appends. We just need
> to
> > implement it. I think this can be done in a way that is fairly low
> overhead
> > and can be "on by default".
> >
> > We actually already provide such a key to the consumer--the offset.
> Making
> > use of this is actually somewhat application dependent. Obviously
> providing
> > exactly-once guarantees in the case of no failures is easy and we already
> > handle that case. The harder part is if a consumer process dies to ensure
> > that it restarts in a position that exactly matches the state changes
> that
> > it has made in some destination system. If the consumer application uses
> > the offset in a way that makes updates idempotent that will work, or if
> > they commit their offset and data atomically that works. However in
> general
> > the goal of a consumer is to produce some state change in another system
> (a
> > db, hdfs, some other data system, etc) and having a general solution that
> > works with all of these is hard since they have very different
> limitations
> > and features.
> >
> > -Jay
> >
> >
> > On Wed, Aug 7, 2013 at 4:00 PM, Yang <teddyyyy...@gmail.com> wrote:
> >
> > > I wonder why at-least-once guarantee is easier to maintain than
> > > exactly-once (in that the latter requires 2PC while the former does
> not ,
> > > according to
> > >
> > >
> >
> http://research.microsoft.com/en-us/um/people/srikanth/netdb11/netdb11papers/netdb11-final12.pdf
> > >  )
> > >
> > > if u achieve at-least-once guarantee, you are able to assert between 2
> > > cases "nothing" vs ">=1 delivered", which can be seen as 2 different
> > > answers 0 and 1. isn't this as hard as the common Byzantine general
> > > problem?
> > >
> > > Thanks
> > > Yang
> > >
> >
>

Reply via email to