kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stevo Slavić <ssla...@gmail.com>
Subject Re: How would Kafka behave in this scenario
Date Tue, 26 Sep 2017 15:21:45 GMT
If I'm not mistaken, kafka-streams exactly once guarantee gives
transactional guarantees, as long as everything happens within single Kafka
cluster.
I.e. logic, based on Kafka streams with exactly once enabled, can read from
that cluster's topic, process read message, optionally write any processing
results to same or typically different topic in the same cluster, and have
commit about successful read also written (to consumer offsets topic) - all
as one transaction, either everything is successful or nothing, you cannot
end up with processing results written to a topic, and exactly same Kafka
message being reread (unless offsets are explicitly reset later or lost for
whatever reason).

So if I understand well, Sameer's dilemma is, when using kafka-streams with
exactly once enabled, a message gets read from the topic, processed by the
logic and processing results are attempted to be published to some topic
(or commit fails) - what will happen if that topic has e.g. min ISR of 2
while actual is 1 ISR, and kafka-streams is configured to publish results
with ack all. Is there any streams specific auto recovery, or does just
regular producer retries apply. If latter, what can one do to
resume/recover from producer failure (e.g. skip, or crash, retry on higher
level)

On Tue, Sep 26, 2017 at 3:02 PM, Ismael Juma <ismael@juma.me.uk> wrote:

> Consumers can fetch messages up to the high watermark, which is dependent
> on the in sync replicas, but not directly dependent on
> `min.insync.replicas` (e.g. if there are 3 in sync replicas, the high
> watermark is the min of the log end offset of the 3 replicas, even if min
> in sync replicas is 2).
>
> Ismael
>
> On Tue, Sep 26, 2017 at 1:34 PM, Denis Bolshakov <
> bolshakov.denis@gmail.com>
> wrote:
>
> > By default kafkf does not allow dirty reads for clients, so while
> > `min.insync.replicas`
> > is not achieved consumers don't see new messages.
> >
> > On 26 September 2017 at 11:09, Sameer Kumar <sam.kum.work@gmail.com>
> > wrote:
> >
> > > Thanks Stevo for pointing me out to correct link.
> > > In this case, how would exactly once feature of streams would behave
> > since
> > > they configure producers with acks=all. I think they would fail and
> would
> > > need to be resumed once the broker comes back.
> > >
> > > -Sameer.
> > >
> > > On Tue, Sep 26, 2017 at 1:09 PM, Stevo Slavić <sslavic@gmail.com>
> wrote:
> > >
> > > > Hello Sameer,
> > > >
> > > > Behavior depends on min.insync.replicas configured for the topic.
> > > > Find more info in the documentation
> > > > https://kafka.apache.org/documentation/#topicconfigs
> > > >
> > > > Kind regards,
> > > > Stevo Slavic.
> > > >
> > > > On Tue, Sep 26, 2017 at 9:01 AM, Sameer Kumar <
> sam.kum.work@gmail.com>
> > > > wrote:
> > > >
> > > > > In case one of the brokers fail,  the broker would get removed from
> > the
> > > > > respective ISR list of those partitions.
> > > > > In case producer has acks=all, how would it behave? would the
> > producers
> > > > be
> > > > > throttled and wait till the broker get backed up.
> > > > >
> > > > > -Sameer.
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > //with Best Regards
> > --Denis Bolshakov
> > e-mail: bolshakov.denis@gmail.com
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message