-
Notifications
You must be signed in to change notification settings - Fork 14k
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-5047: NullPointerException while using GlobalKTable in KafkaStreams #2834
Conversation
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
@@ -174,7 +174,11 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback, | |||
final ConsumerRecords<byte[], byte[]> records = consumer.poll(100); | |||
for (ConsumerRecord<byte[], byte[]> record : records) { | |||
offset = record.offset() + 1; | |||
stateRestoreCallback.restore(record.key(), record.value()); | |||
if (record.key() == null) { | |||
log.warn("null key found at offset %d while restoring state from topic %s and partition %d. Skipping record", record.offset(), record.topic(), record.partition()); |
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.
Could a log level of WARN wreak havoc in case there's a lot of null keys?
I think the log level we pick here should (a) be consistent with what we use elsewhere (not implying it isn't in this PR!) and (b) it should match the guarantees we have documented for global tables. If, for example, we clearly tell the user that null keys are ignored, we could use a less chatty log level than WARN.
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.
ok, i'll make it debug
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 i'm just going to silently drop it as that is what we do elsewhere
stateManager.initialize(context); | ||
final TheStateRestoreCallback stateRestoreCallback = new TheStateRestoreCallback(); | ||
stateManager.register(store1, false, stateRestoreCallback); | ||
assertEquals(1, stateRestoreCallback.restored.size()); |
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.
Why not assertThat(...)
?
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.
because we have to compare KeyValue<byte[], byte[]>
and assertThat()
won't work., i.e., 2 byte[]
with the same contents aren't equal. Need to use assertArrayEquals()
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.
Challenge accepted. ;-)
import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
public class ArrayComparisonTest {
@Test
public void shouldCompareByteArrays() throws Exception {
byte[] first = "foo".getBytes();
byte[] second = "foo".getBytes();
assertThat(first).isEqualTo(second);
}
}
This test passes.
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.
AssertJ knows when it is comparing byte arrays.
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.
We don't use that and i'm not adding another dependency
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.
Oh, sorry -- I thought our assertThat
was from AssertJ.
But using JUnit's/Hamcrest's assertThat
works, too:
import org.junit.Test;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.CoreMatchers.equalTo;
public class ArrayComparisonTest {
@Test
public void shouldCompareByteArrays() throws Exception {
byte[] first = "foo".getBytes();
byte[] second = "foo".getBytes();
assertThat(first, equalTo(second));
}
}
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.
ok -yeah it is actually the KeyValue#equals()
that is causing the problem i was having as it calls byte[].equals
underneath which doesn't work. Anyway, i can work around 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.
Ok. I didn't want to be picky here, it's just that "someone" (cough) kept telling me in my PRs that we should use assertThat
if possible. ;-)
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.
hahaha!
stateManager.register(store1, false, stateRestoreCallback); | ||
assertEquals(1, stateRestoreCallback.restored.size()); | ||
final KeyValue<byte[], byte[]> restoredKv = stateRestoreCallback.restored.get(0); | ||
assertArrayEquals(expectedKey, restoredKv.key); |
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.
Why not assertThat
?
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 above
General comment: We should also update the javadocs of |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
@dguy don't we need to do the same check for the normal KTable too? |
@enothereska For changelogs this will never happen as all key/values that have made it into the changelog will be ok. So restoration is ok. For a |
@dguy (/cc @enothereska): I'd also update the KTable javadocs to cover the null-key behavior/contract. |
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.
LGTM.
Just a general comment: should we add a metric-sensor to report if null
-key records get dropped?
@@ -54,6 +54,8 @@ | |||
*}</pre> | |||
* Note that in contrast to {@link KTable} a {@code GlobalKTable}'s state holds a full copy of the underlying topic, | |||
* thus all keys can be queried locally. | |||
* <p> | |||
* Records from the source topic that have null keys are dropped |
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.
Nit. .
missing at the end.
@@ -51,6 +51,8 @@ | |||
* ReadOnlyKeyValueStore view = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore()); | |||
* view.get(key); | |||
*}</pre> | |||
*<p> | |||
* Records from the source topic that have null keys are dropped |
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 above
@mjsax that is a good idea, though i i'll file a JIRA for it as i think it applies more broadly than this particular fix. We drop records in various places and don't record it. |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
@enothereska i updated it to check KTable. Can you please make another pass? @ijuma would you mind having a look once @enothereska has given the thumbs up? |
LGTM thanks. |
@guozhangwang can you please merge? |
LGTM and Merged to trunk. @dguy I tried to cherry-pick to 0.10.2 but the conflicts are quite complex for me to resolve-forward, could you file a separate PR for 0.10.2 also? |
Skip null keys when initializing GlobalKTables. This is inline with what happens during normal processing.