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

Stanislav Spiridonov edited comment on KAFKA-16382 at 3/27/24 8:57 AM:
-----------------------------------------------------------------------

Wait, why is it not a bug? I have the real scenario. 

Good scenario:
 # Some events happened. Output topic contains mapped result.
 # Event ends ({_}*null*{_} body).  Output topic contains delete message for 
the event. 

Wrong scenario:
 # Some events happened. Output topic contains mapped result.
 # We stopped Kafka, Make some updates + {*}full rest{*}. 
 # Meanwhile event ends ({*}_null_{*} body).  
 # We start Kafka, it process the input topic from scratch but "optimise" 
internally nulls. The output topic *still* contains mapped result. The delete 
message *never* reach the output topic.

As work around we can clear the output topic before Kafka start. But in out 
system the output topic is input for another Kafka Stream application so we 
need to reset all subsequent Kafka Stream applications to correct this 
behaviour.

Another workaround is on each delete message (null body) generate synthetic 
message with synthetic key and some value in the body (it will not optimised) 
and check for such messages on write to output topic and generate back the 
delete message to the output, but it is also looks as a hack.

 


was (Author: foal):
Wait, why is it not a bug? I have the real scenario. 

Good scenario:
 # Some events happened. Output topic contains mapped result.
 # Event ends ({_}*null*{_} body).  Output topic contains delete message for 
the event. 

Wrong scenario:
 # Some events happened. Output topic contains mapped result.
 # We stopped Kafka, Make some updates + {*}full rest{*}. 
 # Meanwhile event ends ({*}_null_{*} body).  
 # We start Kafka, it process the input topic from scratch but "optimise" 
internally nulls. The output topic *still* contains mapped result. The delete 
message *never* reach the output topic.

As work around we can clear the output topic before Kafka start. But in out 
system the output topic is input for another Kafka Stream application so we 
need to reset all subsequent Kafka Stream applications to correct this 
behaviour.

Another workaround is on each null body generate another synthetic message with 
another key and some value in the body (it will not optimised) and check for 
such messages on write to output topic and generate back the delete message, 
but it is also looks as a hack.

 

> Kafka Streams drop NULL values after reset
> ------------------------------------------
>
>                 Key: KAFKA-16382
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16382
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 3.6.1
>            Reporter: Stanislav Spiridonov
>            Priority: Major
>
> Kafka Streams (KTable) drops null values after full reset.
> See 
> [https://github.com/foal/Null-Issue/blob/main/src/main/java/NullProblemExample.java]
>  for sample topology
> Step to reproduce (req NULL-IN, NULL-IN-AUX, NULL-OUT topics)
>  # Start example - 1st round
>  # Send to NULL-IN "A1:a" -> NULL-OUT "A1:anull"
>  # Send to NULL-IN-AUX "A1:b" -> NULL-OUT "A1:anull, A1:ab"
>  # Stop application 
>  # Run kafka-streams-application-reset 
> {code:java}
> call bin/windows/kafka-streams-application-reset --application-id 
> nullproblem-example^
>  --input-topics "NULL-IN,NULL-IN-AUX"^
>  --bootstrap-server "localhost:9092"
> {code}
>  # Send to NULL-IN-AUX "A1:" -> NULL-OUT "A1:anull, A1:ab" - it is Ok (no app 
> running yet)
>  # Start example - 2nd round
>  # After initialization -> NULL-OUT *still contains* "A1:anull, A1:ab"
>  # Expected output "A1:anull, A1:ab, A1:"
> The issue is NOT reproduced if application just restarted (skip step 5). 
> The issue is NOT reproduced if internal cache is disabled.



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

Reply via email to