[ https://issues.apache.org/jira/browse/KAFKA-12323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17283909#comment-17283909 ]
Adam Bellemare commented on KAFKA-12323: ---------------------------------------- Alright, I think I found source of the 0 timestamp assignment: This is returning a 0 timestamp for a punctuation-created event. [https://github.com/apache/kafka/blob/2.7/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java#L169] It calls through to AbstractProcessorContextImpl, which executes this and returns 0L for every punctuated event. [https://github.com/apache/kafka/blob/2.7/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java#L176] I have observed that this will always assign 0L for the timestamp, because *recordContext* is always null for this processor. I noticed that I am adding the processor through a now-deprecated *addProcessor* API call. I suspect that the deprecation change is responsible for the bug: {code:java} topology //This addProcessor call is deprecated. .addProcessor(progressProcessorName, punctuationProcessor(...), sinkEventsName) .addSink("progressSink", "topicName", serdeFactory.serde(true).serializer(), serdeFactory.progressEventSerde().serializer(), progressProcessorName); {code} I have attached the code ([^PunctuateTimestampZeroTest.java] that illustrates the error. You can run it as an integration test in Kafka (you will have to disable warnings-as-errors (Werror) from build.gradle, as it will fail because I am using a deprecated function). I don't think this is a blocker because I suspect I can work around it with the non-deprecated function. It is, however, quite major, as a person can upgrade their code and have this happen without noticing. It's particularly insidious as it may cause downstream tasks that rely on the timestamp to silently fail (as is what happened to me). {color:#cc7832} {color} > Record timestamps not populated in event > ---------------------------------------- > > Key: KAFKA-12323 > URL: https://issues.apache.org/jira/browse/KAFKA-12323 > Project: Kafka > Issue Type: Bug > Affects Versions: 2.7.0 > Reporter: Adam Bellemare > Priority: Blocker > Attachments: PunctuateTimestampZeroTest.java > > > Upgraded a kafka streams application from 2.6.0 to 2.7.0. Noticed that the > events being produced had a "CreatedAt" timestamp = 0, causing downstream > failures as we depend on those timestamps. Reverting back to 2.6.0/2.6.1 > fixed this issue. This was the only change to the Kafka Streams application. > Consuming the event stream produced by 2.6.0 results in events that, when > consumed using the `kafka-avro-console-consumer` and `--property > print.timestamp=true` result in events prepended with the event times, such > as: > {code:java} > CreateTime:1613072202271 <key> <value> > CreateTime:1613072203412 <key> <value> > CreateTime:1613072205431 <key> <value> > {code} > etc. > However, when those events are produced by the Kafka Streams app using 2.7.0, > we get: > {code:java} > CreateTime:0 <key> <value> > CreateTime:0 <key> <value> > CreateTime:0 <key> <value> > {code} > I don't know if these is a default value somewhere that changed, but this is > actually a blocker for our use-cases as we now need to circumnavigate this > limitation (or roll back to 2.6.1, though there are other issues we must deal > with then). I am not sure which unit tests in the code base to look at to > validate this, but I wanted to log this bug now in case someone else has > already seen this or an open one exists (I didn't see one though). -- This message was sent by Atlassian Jira (v8.3.4#803005)