Hi Matthias,

I get these errors even on reprocessing, when data is flowing full throttle
through the system. Can you help me understand how to tune this behavior,
if possible? I appreciate that it's aggressive, but it seems to be so
extremely aggressive that I get these errors constantly. Just how much data
do you need to have flowing before they go away?

Thank you,
Dmitry

On Thu, Dec 20, 2018 at 7:22 AM Matthias J. Sax <matth...@confluent.io>
wrote:

> The problem is repartitions topics:
>
> Kafka Streams considers those topics as transient and purges consumed
> data aggressively (cf https://issues.apache.org/jira/browse/KAFKA-6150)
> resulting in lost producer state for those topics :(
>
>
> -Matthias
>
> On 12/20/18 3:18 AM, Dmitry Minkovsky wrote:
> > Also, I have read through that issue and KIP-360 to the extent my
> knowledge
> > allows and I don't understand why I get this error constantly when
> exactly
> > once is enabled. The KIP says
> >
> >> Idempotent/transactional semantics depend on the broker retaining state
> > for each active producer id (e.g. epoch and sequence number). When the
> > broker loses that state–due to segment deletion or a call to
> > DeleteRecords–then additional produce requests will result in the
> > UNKNOWN_PRODUCER_ID error.
> >
> > How much throughput do you need before this goes away? It seems like this
> > happens for me on every call... with the calls seconds apart.
> >
> > On Wed, Dec 19, 2018 at 9:12 PM Dmitry Minkovsky <dminkov...@gmail.com>
> > wrote:
> >
> >> Hello 王美功,
> >>
> >> I am using 2.1.0. And, I think you nailed it on the head, because my
> >> application is low throughput and I am seeing UNKNOWN_PRODUCER_ID all
> the
> >> time with exactly once enabled. I've googled this before but couldn't
> >> identify the cause. Thank you!
> >>
> >> Setting retry.backoff.ms to 5 brought the latency down from 1.3s to
> >> 750ms. That's tolerable for me, but I am wondering: when the KAFKA-7190
> is
> >> fixed, will the latency drop further?
> >>
> >> Dmitry
> >>
> >> On Wed, Dec 19, 2018 at 8:38 PM meigong.wang <meigong.w...@okcoin.com>
> >> wrote:
> >>
> >>> Which version are you using? This bug(
> >>> https://issues.apache.org/jira/browse/KAFKA-7190) may increase the
> >>> latency of your application, try to reduce the retry.backoff.ms,the
> >>> default value is 100 ms.
> >>>
> >>>
> >>> 王美功
> >>>
> >>>
> >>> 原始邮件
> >>> 发件人:Dmitry minkovskydminkov...@gmail.com
> >>> 收件人:usersus...@kafka.apache.org
> >>> 发送时间:2018年12月20日(周四) 09:25
> >>> 主题:High end-to-end latency with processing.guarantee=exactly_once
> >>>
> >>>
> >>> I have a process that spans several Kafka Streams applications. With
> the
> >>> streams commit interval and producer linger both set to 5ms, when
> exactly
> >>> once delivery is disabled, this process takes ~250ms. With exactly once
> >>> enabled, the same process takes anywhere from 800-1200ms. In Enabling
> >>> Exactly-Once in Kafka Streams ,">
> >>> https://www.confluent.io/blog/enabling-exactly-kafka-streams/,
> Guozhang
> >>> writes In Kafka Streams, because a new transaction is created whenever
> >>> commit is called, the average transaction size is determined by the
> commit
> >>> interval: with the same incoming traffic, a shorter commit interval
> will
> >>> result in smaller transactions. In practice users should therefore
> tune the
> >>> commit.interval.ms setting when exactly-once is enabled to make a good
> >>> trade-off between throughput versus end-to-end processing latency. But
> I am
> >>> not seeing much of a difference when I tune commit.interval.ms with
> >>> exactly once enabled. `though()` and `.to()/.stream()` take 100-250ms
> even
> >>> with commit.interval.ms set to 5ms. Do these latency differences sound
> >>> right? Is something off? Thank you, Dmitry
> >>
> >>
> >
>
>

Reply via email to