vvcephei commented on a change in pull request #11514:
URL: https://github.com/apache/kafka/pull/11514#discussion_r754789750



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java
##########
@@ -62,6 +67,12 @@ public void init(final ProcessorContext context,
         open = true;
     }
 
+    @Override
+    public void init(final StateStoreContext context,
+                     final StateStore root) {
+        this.context = context;

Review comment:
       ```suggestion
           if (root != null) {
               // register the store
               context.register(root, (key, value) -> put(Bytes.wrap(key), 
value));
           }
           open = true;
           this.context = context;
   ```
   
   Hey, @patrickstuedi , I'm sorry, but I couldn't walk away from how weird 
that NPE was. I took another look and figured out it's because we added the 
_new_ init method here, but didn't copy over all the logic from the old init 
(right above). If we don't call `register`, then the store will actually not be 
registered at all, leading to an NPE when anyone tries to look the store up.
   
   This was a miss on my part. I should have migrated this store to the new 
init method already. It's also definitely a sharp edge in the migration from 
the new Processor API (and ProcessorContext above) to the new Processor API 
(which includes the StateStoreContext here).
   
   Here's the KIP in case you want to read up on what's going on with that 
improvement and also specifically why there are two versions of init here: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-478+-+Strongly+typed+Processor+API
   
   Anyway, the patch above fixes it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to