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

Guozhang Wang commented on KAFKA-10688:
---------------------------------------

Without KAFKA-3370, then we have to implement the desired behavior at the 
streams layer itself. That is:

1) Upon task assignment, explicitly set the starting offset for the main 
consumer based on the per-topic / global reset policy. For repartition topics, 
the reset policy would be `latest`.

2) Upon task revive (for corrupted exception handling), do the same thing as 1).

3) During normal processing, if an InvalidOffsetException is thrown from main 
consumer, we differentiate these cases: 3.a) for source topics: log a warning 
and reset accordingly; 3.b) for repartition topics throw as fatal errors.

We can potentially be more strict that we require all topics contains committed 
offset, if only some of them have committed positions then fail. But for 
extensibility I'm going to hold on doing that for now.

> Handle accidental truncation of repartition topics as exceptional failure
> -------------------------------------------------------------------------
>
>                 Key: KAFKA-10688
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10688
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>            Reporter: Guozhang Wang
>            Assignee: Guozhang Wang
>            Priority: Major
>
> Today we always handle InvalidOffsetException from the main consumer by the 
> resetting policy assuming they are for source topics. But repartition topics 
> are also source topics and should never be truncated and hence cause 
> InvalidOffsetException.
> We should differentiate these repartition topics from external source topics 
> and treat the InvalidOffsetException from repartition topics as fatal and 
> close the whole application.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to