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

ASF GitHub Bot commented on KAFKA-6487:
---------------------------------------

bartdevylder opened a new pull request #4495: KAFKA-6487: 
ChangeLoggingKeyValueBytesStore does not propagate delete
URL: https://github.com/apache/kafka/pull/4495
 
 
   The `ChangeLoggingKeyValueBytesStore` used to write null to its underlying 
store instead of propagating the delete, which has two drawbacks:
   - an iterator will see null values
   - unbounded memory growth of the underlying in-memory keyvalue store
   
   The fix will just propagate the delete instead of performing put(key, null). 
   
   The changes to the tests:
   - extra test whether the key is really gone after delete by calling the 
`approximateEntries` on the underlying store. This number is exact because we 
know the underlying store in the test is of type `InMemoryKeyValueStore`
   - extra test to check a delete is logged as <key, null>  (the existing test 
would also succeed if the key is just absent)
   
   While also updating the corresponding tests of the 
`ChangeLoggingKeyValueStore` I noticed the class is nowhere used anymore so I 
removed it from the source code for clarity.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> ChangeLoggingKeyValueBytesStore.all() returns null
> --------------------------------------------------
>
>                 Key: KAFKA-6487
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6487
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 1.0.0
>            Reporter: Bill Bejeck
>            Assignee: Bill Bejeck
>            Priority: Major
>
> The  {{ChangeLoggingKeyValueBytesStore}} implements the {{KeyValueStore}} 
> interface which extends the {{ReadOnlyKeyValueStore}} interface.  The Javadoc 
> for {{ReadOnlyKeyValueStore#all}} states the method should never return a 
> {{null}} value.
> But when deleting a record from the {{ChangeLoggingKeyValueBytesStore}} and 
> subsequently calling the {{all}} method, a null value is returned.
>  
> https://issues.apache.org/jira/browse/KAFKA-4750 is a related issue



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to