Hi Lucas,

This was indeed the issue, and your fix resolved it.
Thanks for the assistance — greatly appreciated.
Best,
Yves.

C2 - Restricted use
________________________________
Van: Lucas Brutschy via users <[email protected]>
Verzonden: maandag 27 april 2026 13:46
Aan: [email protected] <[email protected]>
CC: Lucas Brutschy <[email protected]>
Onderwerp: Re: Kafka Streams 4.1.2 offsets not being committed after upgrade 
from Spring Boot 3.5.8 to 4.0.5

Hi Yves,

do you use a consumer interceptor or wrapper that constructs
ConsumerRecord objects using a deprecated pre-4.0 constructor? We have
seen these problems when people use these kinds of outdated overloads;
see 
https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-1094%253A%2BAdd%2Ba%2Bnew%2Bconstructor%2Bmethod%2Bwith%2BnextOffsets%2Bto%2BConsumerRecords&data=05%7C02%7Cyves.stroucken%40soprasteria.com%7Cb1fe40b6d08e47c71e7c08dea452e520%7C8b87af7d86474dc78df45f69a2011bb5%7C0%7C0%7C639128873124923118%7CUnknown%7CTWFpbGZsb3d8eyJFbXB0eU1hcGkiOnRydWUsIlYiOiIwLjAuMDAwMCIsIlAiOiJXaW4zMiIsIkFOIjoiTWFpbCIsIldUIjoyfQ%3D%3D%7C0%7C%7C%7C&sdata=WWkn%2FRCyvo8QVwGpM5QM1A%2BTHe58zXRfrifFaB4V4Ko%3D&reserved=0<https://cwiki.apache.org/confluence/display/KAFKA/KIP-1094%3A+Add+a+new+constructor+method+with+nextOffsets+to+ConsumerRecords>.
If that is the case, the solution is to use the new overload.

Cheers,
Lucas

On Fri, Apr 24, 2026 at 5:58 PM STROUCKEN Yves
<[email protected]> wrote:
>
> Hi Bill,
>
> Thanks for the quick reply.
> Here is some of the output I get with debug logging and 
> kafka-consumer-groups.sh
>
> ./kafka-consumer-groups.sh \
>   --bootstrap-server localhost:9092 \
>   --describe \
>   --group 'MyProcessingApplication'
>
> loging.level.org.apache.kafka.streams.processor.internals.StreamTask: DEBUG
>
> Starting with a clean topic with no events.
>
> output:
> GROUP                                 TOPIC              PARTITION  
> CURRENT-OFFSET  LOG-END-OFFSET  LAG             CONSUMER-ID                   
>                                                                               
>               HOST            CLIENT-ID
> MyProcessingApplication myTopic.0 4          -               0               
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  / hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 3          -               0               
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  / hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 2          -               0               
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  / hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 1          -               0               
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  / hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 0          -               0               
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  / hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
>
> after producing about 1356 events I get this:
>
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_1] Prepared RUNNING task for committing
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_2] Prepared RUNNING task for committing
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_0] Prepared RUNNING task for committing
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_3] Prepared RUNNING task for committing
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_4] Prepared RUNNING task for committing
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_1] Checkpointable offsets {myTopic.0-1=118}
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_1] Finalized commit for RUNNING task with eos false enforce 
> checkpoint false
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_2] Checkpointable offsets {myTopic.0-2=119}
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_2] Finalized commit for RUNNING task with eos false enforce 
> checkpoint false
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_0] Checkpointable offsets {myTopic.0-0=113}
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_0] Finalized commit for RUNNING task with eos false enforce 
> checkpoint false
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_3] Checkpointable offsets {myTopic.0-3=114}
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_3] Finalized commit for RUNNING task with eos false enforce 
> checkpoint false
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_4] Checkpointable offsets {myTopic.0-4=126}
> 2026-04-24 17:24:39.566 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> DEBUG o.a.k.s.p.internals.StreamTask - stream-thread 
> [MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1] 
> task [0_4] Finalized commit for RUNNING task with eos false enforce 
> checkpoint false
>
> and the '  ./kafka-consumer-groups.sh' result again:
> MyProcessingApplication myTopic.0 4          -               288             
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  /hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 3          -               270             
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  /hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 2          -               264             
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  /hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 1          -               276             
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  /hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 0          -               258             
> -               
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer-6a48934d-d93f-41d5-a89b-4faecaaba720
>  /hostIp     
> MyProcessingApplication-509fdb18-f2ec-43f9-a214-50c3d309fde5-StreamThread-1-consumer
>
> As you can see only 'LOG-END-OFFSET' is moving. Current-offset and LAG stay 
> as '-'.
>
> On other environments the other numbers are filled in because they have run 
> with older versions, and then the displayed lag number stays the same.
> For example, below the consumer group shows large offset lag, while 
> processing is in near realtime with only about 60 events a minute.
>
> MyProcessingApplication myTopic.0 5          2194852         2306908         
> 112056          MyProcessingApplication 
> -1fd70c5b-1f45-4a81-97b4-951c3b7ae861-StreamThread-1-consumer-a3176137-2d89-4278-98ae-c90b3e6fdeb7
>  /Host   MyProcessingApplication 
> -1fd70c5b-1f45-4a81-97b4-951c3b7ae861-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 0          2194725         2305917         
> 111192          MyProcessingApplication 
> -d44f97cf-62ed-499f-90aa-2c563301f2e6-StreamThread-1-consumer-6fbf8c49-00b1-407d-be73-a46524bf62ad
>  /Host   MyProcessingApplication 
> -d44f97cf-62ed-499f-90aa-2c563301f2e6-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 1          2302267         2306596         
> 4329            MyProcessingApplication 
> -d6df029b-658e-4e1c-bc96-b1bd1bcc29be-StreamThread-1-consumer-255edc8a-92de-4c93-bf92-68915bd758e1
>  /Host   MyProcessingApplication 
> -d6df029b-658e-4e1c-bc96-b1bd1bcc29be-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 4          2303234         2306341         
> 3107            MyProcessingApplication 
> -0c0f2d0c-d1a7-415c-9b30-5e2f7dc80899-StreamThread-1-consumer-6ccfcdba-2182-4804-a0f4-ef4e361f2e26
>  /Host   MyProcessingApplication 
> -0c0f2d0c-d1a7-415c-9b30-5e2f7dc80899-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 3          2182863         2306026         
> 123163          MyProcessingApplication 
> -0a8893b3-3ca8-4e6b-9164-a271e13bd3b2-StreamThread-1-consumer-185b65b2-7ba8-40d3-8389-533d12b378bb
>  /Host   MyProcessingApplication 
> -0a8893b3-3ca8-4e6b-9164-a271e13bd3b2-StreamThread-1-consumer
> MyProcessingApplication myTopic.0 2          2301468         2306056         
> 4588            MyProcessingApplication 
> -dcb5ee39-0b95-42ce-a83a-44e1e0e0ad76-StreamThread-1-consumer-458d237a-28ec-47ec-9acb-786f5a2d5bc2
>  /Host    MyProcessingApplication 
> -dcb5ee39-0b95-42ce-a83a-44e1e0e0ad76-StreamThread-1-consumer
>
> There are no errors on the current log settings.
>
> I hope this is useful, please let me know if there is anything else I can do 
> to help.
>
> Thanks,
> Yves.
>
>
>
> On 24/04/2026, 16:29, "Bill Bejeck via users" <[email protected] 
> <mailto:[email protected]>> wrote:
>
>
> Sorry to hear about the issues. Can you provide log files to help diagnose
> the problem?
>
>
> Thanks,
> Bill
>
>
> On Fri, Apr 24, 2026 at 4:53 AM STROUCKEN Yves <
> [email protected] <mailto:[email protected]>> wrote:
>
>
> > Hello Kafka users,
> > I am looking for help with a Kafka Streams offset commit issue that
> > started after upgrading.
> > We have a Spring Cloud Stream application using the Kafka Streams binder.
> > Old working stack:
> >
> > * Spring Boot 3.5.9
> > * Spring Cloud 2025.0.0
> > * spring-cloud-stream-binder-kafka-streams 4.3.0
> > * kafka-streams 3.9.1
> > New stack with the issue:
> >
> > * Spring Boot 4.0.5
> > * Spring Cloud 2025.1.1
> > * spring-cloud-stream-binder-kafka-streams 5.0.1
> > * kafka-streams 4.1.2
> > Broker version:
> >
> > * Kafka brokers 3.9.0
> > Symptoms:
> >
> > * The Kafka Streams application processes new records normally.
> > * However, committed offsets for the Streams application do not appear
> > to advance.
> > * In Kafka UI, consumer group lag stays high for all partitions, while
> > application-level metrics show near real-time processing of new events.
> > * On restart, the application starts reading from the beginning of the
> > topic again.
> > * When I set auto.offset.reset=none, startup fails with:
> > org.apache.kafka.streams.errors.StreamsException: No valid committed
> > offset found for input [myTopic] and no valid reset policy configured
> > Important notes:
> >
> > * application.id is explicitly configured and static.
> > * The same topology and configuration on the older stack works fine
> > and retains offsets across restart.
> > I have only a basic familiarity with the Kafka Streams internals, but
> > while debugging StreamTask in 4.1.2, I see commit paths being reached even
> > though no actual offsets seem to be committed.
> >
> > Has anyone seen this in 4.1.x, or does this match any known issue?
> > Any pointers, similar bug reports, or things I should inspect next would
> > be greatly appreciated.
> > Thanks
> >
> >
> >
> >
>
>
>

Reply via email to