You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Pushkar Deole <pd...@gmail.com> on 2020/04/24 06:59:40 UTC

Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Hello All,

While using kafka streams application, we are intermittently getting
following exception and stream is closed. We need to restart the
application to get it working again and start processing. This exception is
observed in some of the labs which are being idle for some time but it is
not observed always. Any inputs appreciated here.

{"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
[analytics-event-filter-StreamThread-1] Failed to commit stream task 2_14
due to the following
error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
Unexpected error in AddOffsetsToTxnResponse: The producer attempted to use
a producer id which is not currently assigned to its transactional
id.\n\tat
org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
java.base/java.lang.Thread.run(Unknown Source)\n"}

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by "Matthias J. Sax" <mj...@apache.org>.
If KafkaStreams goes into ERROR state, you cannot do anything but
restart it.

We constantly improve KafkaStreams to avoid getting into ERROR state but
it's not always possible to auto-recover.

The point being: You should try to figure out the root cause for the
issue (not easy to do unfortunately). Maybe you could change some
configs, but it could also be a bug in KafkaStreams or the broker.

There are more improvement in Apache Kafka 2.6.0 for EOS though. So
maybe it's already fixed there.


-Matthias

On 9/9/20 10:07 PM, Pushkar Deole wrote:
> Matthias,
> 
> Is there any work around after the stream goes into error because of above
> issue like attaching a StateListener on the StreamBuilder and restart the
> stream in case of ERROR state?
> Right now, we need to start the pod that hosts the application which won't
> be feasible when the application goes into production.
> 
> On Thu, Sep 10, 2020 at 2:20 AM Matthias J. Sax <mj...@apache.org> wrote:
> 
>> Well, it's for sure EOS related, but it seems to be a different root cause.
>>
>> I am not aware of any related bug.
>>
>> -Matthias
>>
>>
>> On 9/9/20 4:29 AM, Pushkar Deole wrote:
>>> Hi Matthias,
>>>
>>> We are using confluent kafka and upgraded to confluent version 5.5.0
>> which
>>> I believe maps to 2.5.0 of apache kafka. We tested on few labs by keeping
>>> the solution idle for few days and didn't observe the issue.
>>>
>>> However on one of the labs we observed issue again recently, this is the
>>> exception: unfortunately, don't have complete stack trace.
>>> Anyway, do you think it is same exception as above or is it different?
>> and
>>> whether this is also a kafka server issue that is being reported already?
>>>
>>>
>> {"@timestamp":"2020-08-12T09:17:35.270+00:00","@version":"1","message":"Unexpected
>>> exception in stream
>>>
>> processing.","logger_name":"com.avaya.analytics.filter.ApplicationConfig","thread_name":"analytics-event-filter-Str
>>>
>> eamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.errors.InvalidPidMappingException:
>>> The producer attempted to use a producer id which is not currently
>> assigned
>>> to its transactiona l id.\n"}
>>>
>>> On Sat, May 9, 2020 at 12:49 AM Matthias J. Sax <mj...@apache.org>
>> wrote:
>>>
>>>>>> So does this issue relate to transactions which are used only when
>>>>>> exactly_once guarantee is set?
>>>>
>>>> Correct.
>>>>
>>>> On 5/8/20 6:28 AM, Pushkar Deole wrote:
>>>>> Hello Matthias,
>>>>>
>>>>> By the way, this error seems to be occurring in only one of the
>> services.
>>>>> There is another service which is also using kafka streams to consumer
>>>> from
>>>>> source, uses processors and then a sink to the output topic, however
>> that
>>>>> service is running fine. The difference is this other service is using
>>>>> at_least_once guarantee while the service in error is exactly once
>>>>> guarantee.
>>>>> So does this issue relate to transactions which are used only when
>>>>> exactly_once guarantee is set?
>>>>>
>>>>> On Mon, Apr 27, 2020 at 12:37 PM Pushkar Deole <pd...@gmail.com>
>>>> wrote:
>>>>>
>>>>>> came across this: seems to be the one
>>>>>> https://issues.apache.org/jira/browse/KAFKA-8710
>>>>>>
>>>>>> On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Thanks... can you point to those improvements/bugs that are fixed in
>>>> 2.5?
>>>>>>>
>>>>>>> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org>
>>>> wrote:
>>>>>>>
>>>>>>>> Well, what you say is correct. However, it's a "bug" in the sense
>> that
>>>>>>>> for some cases the producer does not need to fail, but can
>>>> re-initialize
>>>>>>>> itself automatically. Of course, you can also see this as an
>>>> improvement
>>>>>>>> and not a bug :)
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
>>>>>>>>> version used is 2.3
>>>>>>>>> however, not sure if this is a bug.. after doing some search, came
>>>>>>>> across
>>>>>>>>> following for the reason of this:
>>>>>>>>>
>>>>>>>>> essentially, the transaction coordinator of streams is cleaning up
>>>> the
>>>>>>>>> producer and transaction ids after a certain time interval
>> controller
>>>>>>>> by
>>>>>>>>> transactional.id.expiration.ms
>>>>>>>>> <
>>>>>>>>
>>>>
>> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
>>>>>>>>> ,
>>>>>>>>> if the coordinator doesn't receive any updates/writes from the
>>>>>>>> producer for
>>>>>>>>> that much time. Default of this parameter is 7 days and our labs
>> have
>>>>>>>> been
>>>>>>>>> idle for more than that.
>>>>>>>>>
>>>>>>>>> On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mjsax@apache.org
>>>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> This version are you using?
>>>>>>>>>>
>>>>>>>>>> Couple of broker and client side exactly-once related bugs got fix
>>>> in
>>>>>>>>>> the latest release 2.5.0.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 4/23/20 11:59 PM, Pushkar Deole wrote:
>>>>>>>>>>> Hello All,
>>>>>>>>>>>
>>>>>>>>>>> While using kafka streams application, we are intermittently
>>>> getting
>>>>>>>>>>> following exception and stream is closed. We need to restart the
>>>>>>>>>>> application to get it working again and start processing. This
>>>>>>>> exception
>>>>>>>>>> is
>>>>>>>>>>> observed in some of the labs which are being idle for some time
>> but
>>>>>>>> it is
>>>>>>>>>>> not observed always. Any inputs appreciated here.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
>>>>>>>>>>> [analytics-event-filter-StreamThread-1] Failed to commit stream
>>>> task
>>>>>>>> 2_14
>>>>>>>>>>> due to the following
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
>>>>>>>>>>> Unexpected error in AddOffsetsToTxnResponse: The producer
>> attempted
>>>>>>>> to
>>>>>>>>>> use
>>>>>>>>>>> a producer id which is not currently assigned to its
>> transactional
>>>>>>>>>>> id.\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
>>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>
>> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
>>>>>>>>>>> java.base/java.lang.Thread.run(Unknown Source)\n"}
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>>
> 


Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by Pushkar Deole <pd...@gmail.com>.
Matthias,

Is there any work around after the stream goes into error because of above
issue like attaching a StateListener on the StreamBuilder and restart the
stream in case of ERROR state?
Right now, we need to start the pod that hosts the application which won't
be feasible when the application goes into production.

On Thu, Sep 10, 2020 at 2:20 AM Matthias J. Sax <mj...@apache.org> wrote:

> Well, it's for sure EOS related, but it seems to be a different root cause.
>
> I am not aware of any related bug.
>
> -Matthias
>
>
> On 9/9/20 4:29 AM, Pushkar Deole wrote:
> > Hi Matthias,
> >
> > We are using confluent kafka and upgraded to confluent version 5.5.0
> which
> > I believe maps to 2.5.0 of apache kafka. We tested on few labs by keeping
> > the solution idle for few days and didn't observe the issue.
> >
> > However on one of the labs we observed issue again recently, this is the
> > exception: unfortunately, don't have complete stack trace.
> > Anyway, do you think it is same exception as above or is it different?
> and
> > whether this is also a kafka server issue that is being reported already?
> >
> >
> {"@timestamp":"2020-08-12T09:17:35.270+00:00","@version":"1","message":"Unexpected
> > exception in stream
> >
> processing.","logger_name":"com.avaya.analytics.filter.ApplicationConfig","thread_name":"analytics-event-filter-Str
> >
> eamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.errors.InvalidPidMappingException:
> > The producer attempted to use a producer id which is not currently
> assigned
> > to its transactiona l id.\n"}
> >
> > On Sat, May 9, 2020 at 12:49 AM Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> >>>> So does this issue relate to transactions which are used only when
> >>>> exactly_once guarantee is set?
> >>
> >> Correct.
> >>
> >> On 5/8/20 6:28 AM, Pushkar Deole wrote:
> >>> Hello Matthias,
> >>>
> >>> By the way, this error seems to be occurring in only one of the
> services.
> >>> There is another service which is also using kafka streams to consumer
> >> from
> >>> source, uses processors and then a sink to the output topic, however
> that
> >>> service is running fine. The difference is this other service is using
> >>> at_least_once guarantee while the service in error is exactly once
> >>> guarantee.
> >>> So does this issue relate to transactions which are used only when
> >>> exactly_once guarantee is set?
> >>>
> >>> On Mon, Apr 27, 2020 at 12:37 PM Pushkar Deole <pd...@gmail.com>
> >> wrote:
> >>>
> >>>> came across this: seems to be the one
> >>>> https://issues.apache.org/jira/browse/KAFKA-8710
> >>>>
> >>>> On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com>
> >>>> wrote:
> >>>>
> >>>>> Thanks... can you point to those improvements/bugs that are fixed in
> >> 2.5?
> >>>>>
> >>>>> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org>
> >> wrote:
> >>>>>
> >>>>>> Well, what you say is correct. However, it's a "bug" in the sense
> that
> >>>>>> for some cases the producer does not need to fail, but can
> >> re-initialize
> >>>>>> itself automatically. Of course, you can also see this as an
> >> improvement
> >>>>>> and not a bug :)
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
> >>>>>>> version used is 2.3
> >>>>>>> however, not sure if this is a bug.. after doing some search, came
> >>>>>> across
> >>>>>>> following for the reason of this:
> >>>>>>>
> >>>>>>> essentially, the transaction coordinator of streams is cleaning up
> >> the
> >>>>>>> producer and transaction ids after a certain time interval
> controller
> >>>>>> by
> >>>>>>> transactional.id.expiration.ms
> >>>>>>> <
> >>>>>>
> >>
> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
> >>>>>>> ,
> >>>>>>> if the coordinator doesn't receive any updates/writes from the
> >>>>>> producer for
> >>>>>>> that much time. Default of this parameter is 7 days and our labs
> have
> >>>>>> been
> >>>>>>> idle for more than that.
> >>>>>>>
> >>>>>>> On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mjsax@apache.org
> >
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> This version are you using?
> >>>>>>>>
> >>>>>>>> Couple of broker and client side exactly-once related bugs got fix
> >> in
> >>>>>>>> the latest release 2.5.0.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 4/23/20 11:59 PM, Pushkar Deole wrote:
> >>>>>>>>> Hello All,
> >>>>>>>>>
> >>>>>>>>> While using kafka streams application, we are intermittently
> >> getting
> >>>>>>>>> following exception and stream is closed. We need to restart the
> >>>>>>>>> application to get it working again and start processing. This
> >>>>>> exception
> >>>>>>>> is
> >>>>>>>>> observed in some of the labs which are being idle for some time
> but
> >>>>>> it is
> >>>>>>>>> not observed always. Any inputs appreciated here.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
> >>>>>>>>> [analytics-event-filter-StreamThread-1] Failed to commit stream
> >> task
> >>>>>> 2_14
> >>>>>>>>> due to the following
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
> >>>>>>>>> Unexpected error in AddOffsetsToTxnResponse: The producer
> attempted
> >>>>>> to
> >>>>>>>> use
> >>>>>>>>> a producer id which is not currently assigned to its
> transactional
> >>>>>>>>> id.\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
> >>>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
> >>>>>>>>> java.base/java.lang.Thread.run(Unknown Source)\n"}
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>
> >>
> >>
> >
>
>

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by "Matthias J. Sax" <mj...@apache.org>.
Well, it's for sure EOS related, but it seems to be a different root cause.

I am not aware of any related bug.

-Matthias


On 9/9/20 4:29 AM, Pushkar Deole wrote:
> Hi Matthias,
> 
> We are using confluent kafka and upgraded to confluent version 5.5.0 which
> I believe maps to 2.5.0 of apache kafka. We tested on few labs by keeping
> the solution idle for few days and didn't observe the issue.
> 
> However on one of the labs we observed issue again recently, this is the
> exception: unfortunately, don't have complete stack trace.
> Anyway, do you think it is same exception as above or is it different? and
> whether this is also a kafka server issue that is being reported already?
> 
> {"@timestamp":"2020-08-12T09:17:35.270+00:00","@version":"1","message":"Unexpected
> exception in stream
> processing.","logger_name":"com.avaya.analytics.filter.ApplicationConfig","thread_name":"analytics-event-filter-Str
> eamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.errors.InvalidPidMappingException:
> The producer attempted to use a producer id which is not currently assigned
> to its transactiona l id.\n"}
> 
> On Sat, May 9, 2020 at 12:49 AM Matthias J. Sax <mj...@apache.org> wrote:
> 
>>>> So does this issue relate to transactions which are used only when
>>>> exactly_once guarantee is set?
>>
>> Correct.
>>
>> On 5/8/20 6:28 AM, Pushkar Deole wrote:
>>> Hello Matthias,
>>>
>>> By the way, this error seems to be occurring in only one of the services.
>>> There is another service which is also using kafka streams to consumer
>> from
>>> source, uses processors and then a sink to the output topic, however that
>>> service is running fine. The difference is this other service is using
>>> at_least_once guarantee while the service in error is exactly once
>>> guarantee.
>>> So does this issue relate to transactions which are used only when
>>> exactly_once guarantee is set?
>>>
>>> On Mon, Apr 27, 2020 at 12:37 PM Pushkar Deole <pd...@gmail.com>
>> wrote:
>>>
>>>> came across this: seems to be the one
>>>> https://issues.apache.org/jira/browse/KAFKA-8710
>>>>
>>>> On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com>
>>>> wrote:
>>>>
>>>>> Thanks... can you point to those improvements/bugs that are fixed in
>> 2.5?
>>>>>
>>>>> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org>
>> wrote:
>>>>>
>>>>>> Well, what you say is correct. However, it's a "bug" in the sense that
>>>>>> for some cases the producer does not need to fail, but can
>> re-initialize
>>>>>> itself automatically. Of course, you can also see this as an
>> improvement
>>>>>> and not a bug :)
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
>>>>>>> version used is 2.3
>>>>>>> however, not sure if this is a bug.. after doing some search, came
>>>>>> across
>>>>>>> following for the reason of this:
>>>>>>>
>>>>>>> essentially, the transaction coordinator of streams is cleaning up
>> the
>>>>>>> producer and transaction ids after a certain time interval controller
>>>>>> by
>>>>>>> transactional.id.expiration.ms
>>>>>>> <
>>>>>>
>> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
>>>>>>> ,
>>>>>>> if the coordinator doesn't receive any updates/writes from the
>>>>>> producer for
>>>>>>> that much time. Default of this parameter is 7 days and our labs have
>>>>>> been
>>>>>>> idle for more than that.
>>>>>>>
>>>>>>> On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org>
>>>>>> wrote:
>>>>>>>
>>>>>>>> This version are you using?
>>>>>>>>
>>>>>>>> Couple of broker and client side exactly-once related bugs got fix
>> in
>>>>>>>> the latest release 2.5.0.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 4/23/20 11:59 PM, Pushkar Deole wrote:
>>>>>>>>> Hello All,
>>>>>>>>>
>>>>>>>>> While using kafka streams application, we are intermittently
>> getting
>>>>>>>>> following exception and stream is closed. We need to restart the
>>>>>>>>> application to get it working again and start processing. This
>>>>>> exception
>>>>>>>> is
>>>>>>>>> observed in some of the labs which are being idle for some time but
>>>>>> it is
>>>>>>>>> not observed always. Any inputs appreciated here.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>
>> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
>>>>>>>>> [analytics-event-filter-StreamThread-1] Failed to commit stream
>> task
>>>>>> 2_14
>>>>>>>>> due to the following
>>>>>>>>>
>>>>>>>>
>>>>>>
>> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
>>>>>>>>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted
>>>>>> to
>>>>>>>> use
>>>>>>>>> a producer id which is not currently assigned to its transactional
>>>>>>>>> id.\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
>>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
>>>>>>>>>
>>>>>>>>
>>>>>>
>> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
>>>>>>>>> java.base/java.lang.Thread.run(Unknown Source)\n"}
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>
>>
>>
> 


Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by Pushkar Deole <pd...@gmail.com>.
Hi Matthias,

We are using confluent kafka and upgraded to confluent version 5.5.0 which
I believe maps to 2.5.0 of apache kafka. We tested on few labs by keeping
the solution idle for few days and didn't observe the issue.

However on one of the labs we observed issue again recently, this is the
exception: unfortunately, don't have complete stack trace.
Anyway, do you think it is same exception as above or is it different? and
whether this is also a kafka server issue that is being reported already?

{"@timestamp":"2020-08-12T09:17:35.270+00:00","@version":"1","message":"Unexpected
exception in stream
processing.","logger_name":"com.avaya.analytics.filter.ApplicationConfig","thread_name":"analytics-event-filter-Str
eamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.errors.InvalidPidMappingException:
The producer attempted to use a producer id which is not currently assigned
to its transactiona l id.\n"}

On Sat, May 9, 2020 at 12:49 AM Matthias J. Sax <mj...@apache.org> wrote:

> >> So does this issue relate to transactions which are used only when
> >> exactly_once guarantee is set?
>
> Correct.
>
> On 5/8/20 6:28 AM, Pushkar Deole wrote:
> > Hello Matthias,
> >
> > By the way, this error seems to be occurring in only one of the services.
> > There is another service which is also using kafka streams to consumer
> from
> > source, uses processors and then a sink to the output topic, however that
> > service is running fine. The difference is this other service is using
> > at_least_once guarantee while the service in error is exactly once
> > guarantee.
> > So does this issue relate to transactions which are used only when
> > exactly_once guarantee is set?
> >
> > On Mon, Apr 27, 2020 at 12:37 PM Pushkar Deole <pd...@gmail.com>
> wrote:
> >
> >> came across this: seems to be the one
> >> https://issues.apache.org/jira/browse/KAFKA-8710
> >>
> >> On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com>
> >> wrote:
> >>
> >>> Thanks... can you point to those improvements/bugs that are fixed in
> 2.5?
> >>>
> >>> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org>
> wrote:
> >>>
> >>>> Well, what you say is correct. However, it's a "bug" in the sense that
> >>>> for some cases the producer does not need to fail, but can
> re-initialize
> >>>> itself automatically. Of course, you can also see this as an
> improvement
> >>>> and not a bug :)
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
> >>>>> version used is 2.3
> >>>>> however, not sure if this is a bug.. after doing some search, came
> >>>> across
> >>>>> following for the reason of this:
> >>>>>
> >>>>> essentially, the transaction coordinator of streams is cleaning up
> the
> >>>>> producer and transaction ids after a certain time interval controller
> >>>> by
> >>>>> transactional.id.expiration.ms
> >>>>> <
> >>>>
> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
> >>>>> ,
> >>>>> if the coordinator doesn't receive any updates/writes from the
> >>>> producer for
> >>>>> that much time. Default of this parameter is 7 days and our labs have
> >>>> been
> >>>>> idle for more than that.
> >>>>>
> >>>>> On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org>
> >>>> wrote:
> >>>>>
> >>>>>> This version are you using?
> >>>>>>
> >>>>>> Couple of broker and client side exactly-once related bugs got fix
> in
> >>>>>> the latest release 2.5.0.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 4/23/20 11:59 PM, Pushkar Deole wrote:
> >>>>>>> Hello All,
> >>>>>>>
> >>>>>>> While using kafka streams application, we are intermittently
> getting
> >>>>>>> following exception and stream is closed. We need to restart the
> >>>>>>> application to get it working again and start processing. This
> >>>> exception
> >>>>>> is
> >>>>>>> observed in some of the labs which are being idle for some time but
> >>>> it is
> >>>>>>> not observed always. Any inputs appreciated here.
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
> >>>>>>> [analytics-event-filter-StreamThread-1] Failed to commit stream
> task
> >>>> 2_14
> >>>>>>> due to the following
> >>>>>>>
> >>>>>>
> >>>>
> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
> >>>>>>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted
> >>>> to
> >>>>>> use
> >>>>>>> a producer id which is not currently assigned to its transactional
> >>>>>>> id.\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
> >>>>>>>
> >>>>
> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
> >>>>>>>
> >>>>>>
> >>>>
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
> >>>>>>> java.base/java.lang.Thread.run(Unknown Source)\n"}
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >
>
>

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by "Matthias J. Sax" <mj...@apache.org>.
>> So does this issue relate to transactions which are used only when
>> exactly_once guarantee is set?

Correct.

On 5/8/20 6:28 AM, Pushkar Deole wrote:
> Hello Matthias,
> 
> By the way, this error seems to be occurring in only one of the services.
> There is another service which is also using kafka streams to consumer from
> source, uses processors and then a sink to the output topic, however that
> service is running fine. The difference is this other service is using
> at_least_once guarantee while the service in error is exactly once
> guarantee.
> So does this issue relate to transactions which are used only when
> exactly_once guarantee is set?
> 
> On Mon, Apr 27, 2020 at 12:37 PM Pushkar Deole <pd...@gmail.com> wrote:
> 
>> came across this: seems to be the one
>> https://issues.apache.org/jira/browse/KAFKA-8710
>>
>> On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com>
>> wrote:
>>
>>> Thanks... can you point to those improvements/bugs that are fixed in 2.5?
>>>
>>> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org> wrote:
>>>
>>>> Well, what you say is correct. However, it's a "bug" in the sense that
>>>> for some cases the producer does not need to fail, but can re-initialize
>>>> itself automatically. Of course, you can also see this as an improvement
>>>> and not a bug :)
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
>>>>> version used is 2.3
>>>>> however, not sure if this is a bug.. after doing some search, came
>>>> across
>>>>> following for the reason of this:
>>>>>
>>>>> essentially, the transaction coordinator of streams is cleaning up the
>>>>> producer and transaction ids after a certain time interval controller
>>>> by
>>>>> transactional.id.expiration.ms
>>>>> <
>>>> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
>>>>> ,
>>>>> if the coordinator doesn't receive any updates/writes from the
>>>> producer for
>>>>> that much time. Default of this parameter is 7 days and our labs have
>>>> been
>>>>> idle for more than that.
>>>>>
>>>>> On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org>
>>>> wrote:
>>>>>
>>>>>> This version are you using?
>>>>>>
>>>>>> Couple of broker and client side exactly-once related bugs got fix in
>>>>>> the latest release 2.5.0.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 4/23/20 11:59 PM, Pushkar Deole wrote:
>>>>>>> Hello All,
>>>>>>>
>>>>>>> While using kafka streams application, we are intermittently getting
>>>>>>> following exception and stream is closed. We need to restart the
>>>>>>> application to get it working again and start processing. This
>>>> exception
>>>>>> is
>>>>>>> observed in some of the labs which are being idle for some time but
>>>> it is
>>>>>>> not observed always. Any inputs appreciated here.
>>>>>>>
>>>>>>>
>>>>>>
>>>> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
>>>>>>> [analytics-event-filter-StreamThread-1] Failed to commit stream task
>>>> 2_14
>>>>>>> due to the following
>>>>>>>
>>>>>>
>>>> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
>>>>>>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted
>>>> to
>>>>>> use
>>>>>>> a producer id which is not currently assigned to its transactional
>>>>>>> id.\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
>>>>>>>
>>>> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
>>>>>>>
>>>>>>
>>>> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
>>>>>>> java.base/java.lang.Thread.run(Unknown Source)\n"}
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
> 


Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by Pushkar Deole <pd...@gmail.com>.
Hello Matthias,

By the way, this error seems to be occurring in only one of the services.
There is another service which is also using kafka streams to consumer from
source, uses processors and then a sink to the output topic, however that
service is running fine. The difference is this other service is using
at_least_once guarantee while the service in error is exactly once
guarantee.
So does this issue relate to transactions which are used only when
exactly_once guarantee is set?

On Mon, Apr 27, 2020 at 12:37 PM Pushkar Deole <pd...@gmail.com> wrote:

> came across this: seems to be the one
> https://issues.apache.org/jira/browse/KAFKA-8710
>
> On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com>
> wrote:
>
>> Thanks... can you point to those improvements/bugs that are fixed in 2.5?
>>
>> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org> wrote:
>>
>>> Well, what you say is correct. However, it's a "bug" in the sense that
>>> for some cases the producer does not need to fail, but can re-initialize
>>> itself automatically. Of course, you can also see this as an improvement
>>> and not a bug :)
>>>
>>>
>>> -Matthias
>>>
>>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
>>> > version used is 2.3
>>> > however, not sure if this is a bug.. after doing some search, came
>>> across
>>> > following for the reason of this:
>>> >
>>> > essentially, the transaction coordinator of streams is cleaning up the
>>> > producer and transaction ids after a certain time interval controller
>>> by
>>> > transactional.id.expiration.ms
>>> > <
>>> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
>>> >,
>>> > if the coordinator doesn't receive any updates/writes from the
>>> producer for
>>> > that much time. Default of this parameter is 7 days and our labs have
>>> been
>>> > idle for more than that.
>>> >
>>> > On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org>
>>> wrote:
>>> >
>>> >> This version are you using?
>>> >>
>>> >> Couple of broker and client side exactly-once related bugs got fix in
>>> >> the latest release 2.5.0.
>>> >>
>>> >>
>>> >> -Matthias
>>> >>
>>> >> On 4/23/20 11:59 PM, Pushkar Deole wrote:
>>> >>> Hello All,
>>> >>>
>>> >>> While using kafka streams application, we are intermittently getting
>>> >>> following exception and stream is closed. We need to restart the
>>> >>> application to get it working again and start processing. This
>>> exception
>>> >> is
>>> >>> observed in some of the labs which are being idle for some time but
>>> it is
>>> >>> not observed always. Any inputs appreciated here.
>>> >>>
>>> >>>
>>> >>
>>> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
>>> >>> [analytics-event-filter-StreamThread-1] Failed to commit stream task
>>> 2_14
>>> >>> due to the following
>>> >>>
>>> >>
>>> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
>>> >>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted
>>> to
>>> >> use
>>> >>> a producer id which is not currently assigned to its transactional
>>> >>> id.\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
>>> >>>
>>> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
>>> >>>
>>> >>
>>> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
>>> >>> java.base/java.lang.Thread.run(Unknown Source)\n"}
>>> >>>
>>> >>
>>> >>
>>> >
>>>
>>>

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by Pushkar Deole <pd...@gmail.com>.
came across this: seems to be the one
https://issues.apache.org/jira/browse/KAFKA-8710

On Mon, Apr 27, 2020 at 12:17 PM Pushkar Deole <pd...@gmail.com> wrote:

> Thanks... can you point to those improvements/bugs that are fixed in 2.5?
>
> On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org> wrote:
>
>> Well, what you say is correct. However, it's a "bug" in the sense that
>> for some cases the producer does not need to fail, but can re-initialize
>> itself automatically. Of course, you can also see this as an improvement
>> and not a bug :)
>>
>>
>> -Matthias
>>
>> On 4/25/20 7:48 AM, Pushkar Deole wrote:
>> > version used is 2.3
>> > however, not sure if this is a bug.. after doing some search, came
>> across
>> > following for the reason of this:
>> >
>> > essentially, the transaction coordinator of streams is cleaning up the
>> > producer and transaction ids after a certain time interval controller by
>> > transactional.id.expiration.ms
>> > <
>> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
>> >,
>> > if the coordinator doesn't receive any updates/writes from the producer
>> for
>> > that much time. Default of this parameter is 7 days and our labs have
>> been
>> > idle for more than that.
>> >
>> > On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org>
>> wrote:
>> >
>> >> This version are you using?
>> >>
>> >> Couple of broker and client side exactly-once related bugs got fix in
>> >> the latest release 2.5.0.
>> >>
>> >>
>> >> -Matthias
>> >>
>> >> On 4/23/20 11:59 PM, Pushkar Deole wrote:
>> >>> Hello All,
>> >>>
>> >>> While using kafka streams application, we are intermittently getting
>> >>> following exception and stream is closed. We need to restart the
>> >>> application to get it working again and start processing. This
>> exception
>> >> is
>> >>> observed in some of the labs which are being idle for some time but
>> it is
>> >>> not observed always. Any inputs appreciated here.
>> >>>
>> >>>
>> >>
>> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
>> >>> [analytics-event-filter-StreamThread-1] Failed to commit stream task
>> 2_14
>> >>> due to the following
>> >>>
>> >>
>> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
>> >>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted to
>> >> use
>> >>> a producer id which is not currently assigned to its transactional
>> >>> id.\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
>> >>>
>> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
>> >>>
>> >>
>> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
>> >>> java.base/java.lang.Thread.run(Unknown Source)\n"}
>> >>>
>> >>
>> >>
>> >
>>
>>

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by Pushkar Deole <pd...@gmail.com>.
Thanks... can you point to those improvements/bugs that are fixed in 2.5?

On Mon, Apr 27, 2020 at 1:03 AM Matthias J. Sax <mj...@apache.org> wrote:

> Well, what you say is correct. However, it's a "bug" in the sense that
> for some cases the producer does not need to fail, but can re-initialize
> itself automatically. Of course, you can also see this as an improvement
> and not a bug :)
>
>
> -Matthias
>
> On 4/25/20 7:48 AM, Pushkar Deole wrote:
> > version used is 2.3
> > however, not sure if this is a bug.. after doing some search, came across
> > following for the reason of this:
> >
> > essentially, the transaction coordinator of streams is cleaning up the
> > producer and transaction ids after a certain time interval controller by
> > transactional.id.expiration.ms
> > <
> https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms
> >,
> > if the coordinator doesn't receive any updates/writes from the producer
> for
> > that much time. Default of this parameter is 7 days and our labs have
> been
> > idle for more than that.
> >
> > On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> >> This version are you using?
> >>
> >> Couple of broker and client side exactly-once related bugs got fix in
> >> the latest release 2.5.0.
> >>
> >>
> >> -Matthias
> >>
> >> On 4/23/20 11:59 PM, Pushkar Deole wrote:
> >>> Hello All,
> >>>
> >>> While using kafka streams application, we are intermittently getting
> >>> following exception and stream is closed. We need to restart the
> >>> application to get it working again and start processing. This
> exception
> >> is
> >>> observed in some of the labs which are being idle for some time but it
> is
> >>> not observed always. Any inputs appreciated here.
> >>>
> >>>
> >>
> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
> >>> [analytics-event-filter-StreamThread-1] Failed to commit stream task
> 2_14
> >>> due to the following
> >>>
> >>
> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
> >>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted to
> >> use
> >>> a producer id which is not currently assigned to its transactional
> >>> id.\n\tat
> >>>
> >>
> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
> >>>
> >>
> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
> >>>
> >>
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
> >>>
> >>
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
> >>>
> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
> >>>
> >>
> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
> >>>
> >>
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
> >>>
> >>
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
> >>> java.base/java.lang.Thread.run(Unknown Source)\n"}
> >>>
> >>
> >>
> >
>
>

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by "Matthias J. Sax" <mj...@apache.org>.
Well, what you say is correct. However, it's a "bug" in the sense that
for some cases the producer does not need to fail, but can re-initialize
itself automatically. Of course, you can also see this as an improvement
and not a bug :)


-Matthias

On 4/25/20 7:48 AM, Pushkar Deole wrote:
> version used is 2.3
> however, not sure if this is a bug.. after doing some search, came across
> following for the reason of this:
> 
> essentially, the transaction coordinator of streams is cleaning up the
> producer and transaction ids after a certain time interval controller by
> transactional.id.expiration.ms
> <https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms>,
> if the coordinator doesn't receive any updates/writes from the producer for
> that much time. Default of this parameter is 7 days and our labs have been
> idle for more than that.
> 
> On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org> wrote:
> 
>> This version are you using?
>>
>> Couple of broker and client side exactly-once related bugs got fix in
>> the latest release 2.5.0.
>>
>>
>> -Matthias
>>
>> On 4/23/20 11:59 PM, Pushkar Deole wrote:
>>> Hello All,
>>>
>>> While using kafka streams application, we are intermittently getting
>>> following exception and stream is closed. We need to restart the
>>> application to get it working again and start processing. This exception
>> is
>>> observed in some of the labs which are being idle for some time but it is
>>> not observed always. Any inputs appreciated here.
>>>
>>>
>> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
>>> [analytics-event-filter-StreamThread-1] Failed to commit stream task 2_14
>>> due to the following
>>>
>> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
>>> Unexpected error in AddOffsetsToTxnResponse: The producer attempted to
>> use
>>> a producer id which is not currently assigned to its transactional
>>> id.\n\tat
>>>
>> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
>>>
>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
>>>
>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
>>>
>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
>>> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
>>>
>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
>>>
>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
>>>
>> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
>>> java.base/java.lang.Thread.run(Unknown Source)\n"}
>>>
>>
>>
> 


Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by Pushkar Deole <pd...@gmail.com>.
version used is 2.3
however, not sure if this is a bug.. after doing some search, came across
following for the reason of this:

essentially, the transaction coordinator of streams is cleaning up the
producer and transaction ids after a certain time interval controller by
transactional.id.expiration.ms
<https://docs.confluent.io/current/installation/configuration/broker-configs.html#transactional-id-expiration-ms>,
if the coordinator doesn't receive any updates/writes from the producer for
that much time. Default of this parameter is 7 days and our labs have been
idle for more than that.

On Fri, Apr 24, 2020 at 10:46 PM Matthias J. Sax <mj...@apache.org> wrote:

> This version are you using?
>
> Couple of broker and client side exactly-once related bugs got fix in
> the latest release 2.5.0.
>
>
> -Matthias
>
> On 4/23/20 11:59 PM, Pushkar Deole wrote:
> > Hello All,
> >
> > While using kafka streams application, we are intermittently getting
> > following exception and stream is closed. We need to restart the
> > application to get it working again and start processing. This exception
> is
> > observed in some of the labs which are being idle for some time but it is
> > not observed always. Any inputs appreciated here.
> >
> >
> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
> > [analytics-event-filter-StreamThread-1] Failed to commit stream task 2_14
> > due to the following
> >
> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
> > Unexpected error in AddOffsetsToTxnResponse: The producer attempted to
> use
> > a producer id which is not currently assigned to its transactional
> > id.\n\tat
> >
> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
> >
> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
> >
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
> >
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
> > org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
> >
> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
> >
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
> >
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
> > java.base/java.lang.Thread.run(Unknown Source)\n"}
> >
>
>

Re: Error in kafka streams: The producer attempted to use a producer id which is not currently assigned to its transactional id

Posted by "Matthias J. Sax" <mj...@apache.org>.
This version are you using?

Couple of broker and client side exactly-once related bugs got fix in
the latest release 2.5.0.


-Matthias

On 4/23/20 11:59 PM, Pushkar Deole wrote:
> Hello All,
> 
> While using kafka streams application, we are intermittently getting
> following exception and stream is closed. We need to restart the
> application to get it working again and start processing. This exception is
> observed in some of the labs which are being idle for some time but it is
> not observed always. Any inputs appreciated here.
> 
> {"@timestamp":"2020-04-15T13:53:52.698+00:00","@version":"1","message":"stream-thread
> [analytics-event-filter-StreamThread-1] Failed to commit stream task 2_14
> due to the following
> error:","logger_name":"org.apache.kafka.streams.processor.internals.AssignedStreamsTasks","thread_name":"analytics-event-filter-StreamThread-1","level":"ERROR","level_value":40000,"stack_trace":"org.apache.kafka.common.KafkaException:
> Unexpected error in AddOffsetsToTxnResponse: The producer attempted to use
> a producer id which is not currently assigned to its transactional
> id.\n\tat
> org.apache.kafka.clients.producer.internals.TransactionManager$AddOffsetsToTxnHandler.handleResponse(TransactionManager.java:1406)\n\tat
> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1069)\n\tat
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)\n\tat
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:561)\n\tat
> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:553)\n\tat
> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425)\n\tat
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:311)\n\tat
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)\n\tat
> java.base/java.lang.Thread.run(Unknown Source)\n"}
>