New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-12396 added a nullcheck before trying to retrieve a key #10548
Conversation
LGTM |
@@ -326,6 +326,7 @@ public void putAll(final List<KeyValue<Bytes, byte[]>> entries) { | |||
|
|||
@Override | |||
public synchronized byte[] get(final Bytes key) { | |||
Objects.requireNonNull(key, "key cannot be null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we should add this to MeteredKeyValueStore
instead -- this way, we get the same guard for the in-memory key-value store (or other custom stores).
We should also add this check to other methods that accept a key, like put
, putIfAbsent
etc...
Furthermore, we should also add it for other store types, ie, MeteredTimestampedKeyValueStore
, MeteredWindowedStore
, MeteredTimestampedWindowStore
and MeteredSessionStore
.
Last, but not least, we should add corresponding unit tests.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
On it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This check can be remove, as we do the check in the "metered" store that will wrap this one.
…test coverage for aforementioned nullchecks
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A few methods are missing:
MeteredKeyValueStore:
- putAll() (we should iterator over all entries to check if key might be
null
- prefixScan() (prefix should not be
null
either I guess? \cc @guozhangwang @cadonna - range() (both keys)
- reverseRange (both keys)
MeteredSessionStore:
- put() (should verify
sessionKey != null
andsessionKey.key() != null
) - remove()
- fetchSession()
- fetch(K)
- backwardFetch(K)
- fetch(K, K)
- backwardFetch(K, K)
- findSession(K, long, long)
- backwardFindSession
- findSession(K, K, long, long)
- backwardFindSession(K, K, long, long)
Thanks for adding tests!
@@ -290,6 +295,7 @@ protected V outerValue(final byte[] value) { | |||
} | |||
|
|||
protected Bytes keyBytes(final K key) { | |||
Objects.requireNonNull(key, "key cannot be null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method is not public
so no need to add this check
@@ -59,6 +61,7 @@ | |||
|
|||
|
|||
public RawAndDeserializedValue<V> getWithBinary(final K key) { | |||
Objects.requireNonNull(key, "key cannot be null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually not public API -- don't need this check.
@@ -73,6 +76,7 @@ | |||
public boolean putIfDifferentValues(final K key, | |||
final ValueAndTimestamp<V> newValue, | |||
final byte[] oldSerializedValue) { | |||
Objects.requireNonNull(key, "key cannot be null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually not public API -- don't need this check.
@@ -326,6 +326,7 @@ public void putAll(final List<KeyValue<Bytes, byte[]>> entries) { | |||
|
|||
@Override | |||
public synchronized byte[] get(final Bytes key) { | |||
Objects.requireNonNull(key, "key cannot be null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This check can be remove, as we do the check in the "metered" store that will wrap this one.
…MeteredTimestampedKeyValueStore, added new checks to MeteredKeyValueStore, covered new checks and old checks for MeteredSessionStoreTest
All of them already had the checks. Some lacked test coverage, though. I made changes according to your comments and added some tests. |
Sweet. I did not double check the code before. Seems, |
Yes! Actually, the implementations in
I think it would make sense to move them to the |
Tests for aforementioned checks were added here: |
The input parameter is |
Ooohhhh. I see. Yea, sorry, I also completely overlooked the wrapped key part. I'll fix it. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for staying on top of this! Seems we find more corner case while reviewing... A few last comments.
@@ -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 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we could simplify this to a one liner?
entries.forEach(entry -> Objects.requireNonNull(entry.key, "key cannot be null"));
@@ -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"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
KafkaStreams runtime always "wraps" any store with a corresponding MeteredXxxStore
(cf https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreBuilder.java#L42-L49) -- those MeteredXxxStores
do the transaction from objects to bytes (ie they use the serdes) and also track state store metrics. (Note that stores provided to the runtime always have type <Bytes, byte[]>
while they are exposed to Processors
as <K,V>
types.)
Thus, when you call context.stateStore(...)
you always get a MeteredXxxStore
object -- of course, those details are hidden behind the interface type.
This architecture allows us to unify code and separate concerns. In fact, it also allows us to add/remove more "layers": we can insert a "caching layer" (cf. https://kafka.apache.org/28/documentation/streams/developer-guide/memory-mgmt.html) and a "change logging layer" (both are inserted by default).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oooooh, I see.
@@ -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")); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This test remind me, that the SessionWindow
what is wrapped should not be null
either.
…toring in MeteredKeyValueStore\InMemoryKeyValueStore\RocksDBStoreTest
Seems there is some checkstyle error:
|
@@ -402,6 +402,11 @@ public void shouldReturnKeysWithGivenPrefix() { | |||
assertThat(valuesWithPrefix.get(2), is("b")); | |||
} | |||
|
|||
@Test | |||
public void shouldThrowNullPointerIfPrefixKeySerializerIsNull() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This test fails now, as the check was removed. No need to add this test any longer in this class.
To avoid checkstyle and test issues, and reduce review turn-around time, I would recommend to run unit tests and checkstyle locally before pushing update: |
Yeah, sorry, I forget to do that sometimes. |
Thanks for the PR @Nathan22177! Merged to |
No description provided.