[ 
https://issues.apache.org/jira/browse/KAFKA-14294?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17630678#comment-17630678
 ] 

A. Sophie Blee-Goldman commented on KAFKA-14294:
------------------------------------------------

I guess we should just remove that check? ie first go through all the tasks and 
check whether any of them have commitNeeded = true, and if so we still need to 
commit whether or not there are actually "new" offsets to be committed.

 

I do worry a bit about the potential impact of excessively committing, maybe we 
should try to expose/check whether the producer has an open transaction OR 
there are offsets to commit, rather than relying on the commitNeeded flag which 
would be true regardless of whether the punctuation actually tried to forward 
new records/open a new transaction

> Kafka Streams should commit transaction when no records are processed
> ---------------------------------------------------------------------
>
>                 Key: KAFKA-14294
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14294
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>    Affects Versions: 3.2.1
>            Reporter: Vicky Papavasileiou
>            Priority: Major
>
> Currently, if there are no records to process in the input topic, a 
> transaction does not commit. If a custom punctuator code is writing to a 
> state store (which is common practice) the producer gets fenced when trying 
> to write to the changelog topic. This throws a TaskMigratedException and 
> causes a rebalance. 
> A better approach would be to commit a transaction even when there are no 
> records processed as to allow the punctuator to make progress. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to