[ https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16062112#comment-16062112 ]
Matthias J. Sax commented on KAFKA-4750: ---------------------------------------- Your observation is correct. I simplified the call chain. The question than is, it is a bug in the {{Serde}} that it does not return {{null}} if value is {{null}}? Or should we just make Streams itself more robust and change {{byte[] rawValue = serdes.rawValue(value);}} to {{byte[] rawValue = value == null ? null : serdes.rawValue(value);}} ? > KeyValueIterator returns null values > ------------------------------------ > > Key: KAFKA-4750 > URL: https://issues.apache.org/jira/browse/KAFKA-4750 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1 > Reporter: Michal Borowiecki > Assignee: Evgeny Veretennikov > Labels: newbie > Attachments: DeleteTest.java > > > The API for ReadOnlyKeyValueStore.range method promises the returned iterator > will not return null values. However, after upgrading from 0.10.0.0 to > 0.10.1.1 we found null values are returned causing NPEs on our side. > I found this happens after removing entries from the store and I found > resemblance to SAMZA-94 defect. The problem seems to be as it was there, when > deleting entries and having a serializer that does not return null when null > is passed in, the state store doesn't actually delete that key/value pair but > the iterator will return null value for that key. > When I modified our serilizer to return null when null is passed in, the > problem went away. However, I believe this should be fixed in kafka streams, > perhaps with a similar approach as SAMZA-94. -- This message was sent by Atlassian JIRA (v6.4.14#64029)