David Mao created KAFKA-15344:
---------------------------------
Summary: Kafka Streams should include the message leader epoch
when committing offsets
Key: KAFKA-15344
URL: https://issues.apache.org/jira/browse/KAFKA-15344
Project: Kafka
Issue Type: Bug
Reporter: David Mao
We noticed an application received an OFFSET_OUT_OF_RANGE error following a
network partition and streams task rebalance and subsequently reset its offsets
to the beginning.
Inspecting the logs, we saw multiple consumer log messages like:
{code:java}
Setting offset for partition tp to the committed offset
FetchPosition{offset=1234, offsetEpoch=Optional.empty...)
{code}
Inspecting the streams code, it looks like kafka streams calls `commitSync`
passing through an explicit OffsetAndMetadata object but does not populate the
offset leader epoch.
The offset leader epoch is required in the offset commit to ensure that all
consumers in the consumer group have coherent metadata before fetching.
Otherwise after a consumer group rebalance, a consumer may fetch with a stale
leader epoch with respect to the committed offset and get an offset out of
range error from a zombie partition leader.
The low-hanging fruit fix would be to have streams pass in the message epoch
for each commit. Another fix discussed with [~hachikuji] is to have the
consumer cache leader epoch ranges, similar to how the broker maintains a
leader epoch cache.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)