mjsax commented on a change in pull request #10548:
URL: https://github.com/apache/kafka/pull/10548#discussion_r618760487



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -219,11 +224,19 @@ public V putIfAbsent(final K key,
 
     @Override
     public void putAll(final List<KeyValue<K, V>> entries) {
+        final List<KeyValue<K, V>> possiblyNullKeys = entries

Review comment:
       I think we could simplify this to a one liner?
   ```
   entries.forEach(entry -> Objects.requireNonNull(entry.key, "key cannot be 
null"));
   ```

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java
##########
@@ -472,11 +472,26 @@ public void shouldThrowNullPointerOnRemoveIfKeyIsNull() {
         assertThrows(NullPointerException.class, () -> store.remove(null));
     }
 
+    @Test
+    public void shouldThrowNullPointerOnPutIfWrappedKeyIsNull() {
+        assertThrows(NullPointerException.class, () -> store.put(new 
Windowed<>(null, new SessionWindow(0, 0)), "a"));

Review comment:
       This test remind me, that the `SessionWindow` what is wrapped should not 
be `null` either. 

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -234,13 +247,15 @@ public V delete(final K key) {
 
     @Override
     public <PS extends Serializer<P>, P> KeyValueIterator<K, V> 
prefixScan(final P prefix, final PS prefixKeySerializer) {
-
+        Objects.requireNonNull(prefix, "key cannot be null");

Review comment:
       As mentioned by @cadonna the wrapped stores, also check 
`prefixKeySerializer` for null -- thus might be good to move both check here.
   
   I think we can also remove both checks in `RocksDBStore` and 
`InMemoryKeyValueStore` -- they seems to be redundant now? 




-- 
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.

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


Reply via email to