Thanks Colin, it makes sense now, it was the HWM party I was missing.

Cheers,

Tom

On 6 Oct 2017 6:44 pm, "Colin McCabe" <cmcc...@apache.org> wrote:

On Thu, Oct 5, 2017, at 12:06, Tom Bentley wrote:
> Hi Colin,
>
> Is it really true that "the period when the offset is unavailable should
> be
> brief"? I'm thinking about a producer with acks=1, so the old leader
> returns the ProduceResponse immediately and then is replaced before it
> can
> sent a FetchResponse to any followers.
> The new leader is then waiting for
> more messages from producers in order for its high watermark to increase
> (because it's log doesn't have the original messages in, so its HW can't
> catch up with this). This wait could be be arbitrarily long.

Hi Tom,

As I understand it, the sequence of events you are proposing is this:

1. Producer with acks=1 sends message batch B to node 1 (current
leader).
2. Node 1 fails before any replication can take place
3. Node 2 becomes the leader.

In this scenario, Node 2's log end offset (LEO) does not include message
batch B.  So there is no wait (or at least, no wait that is due to batch
B).

Also, Node 1 cannot advance its high water mark (HWM) until the replicas
have caught up.  So the HWM never goes backwards.  Batch B simply
disappears without a trace -- no consumers ever were able to consume it,
and it never advanced the partition HWM.  That's life with acks=1.

cheers,
Colin

>
> I rather suspect this isn't a problem really and that I misunderstand the
> precise details of the protocol, but it would be beneficial to me to
> discover my misconceptions.
>
> Thanks,
>
> Tom
>
>
>
> On 5 October 2017 at 19:23, Colin McCabe <cmcc...@apache.org> wrote:
>
> > Hi all,
> >
> > I created a KIP for discussion about fixing a corner case in
> > ListOffsetsResponse.  Check it out at:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 207%3A+Offsets+returned+by+ListOffsetsResponse+should+be+
> > monotonically+increasing+even+during+a+partition+leader+change
> >
> > cheers,
> > Colin
> >

Reply via email to