Skip to content
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-8731: InMemorySessionStore throws NullPointerException on startup #7132

Merged
merged 2 commits into from Jul 31, 2019

Conversation

@ableegoldman
Copy link
Contributor

commented Jul 29, 2019

Should be cherry-picked to 2.3

@ableegoldman

This comment has been minimized.

Copy link
Contributor Author

commented Jul 29, 2019

@@ -120,7 +120,15 @@ public void put(final Windowed<Bytes> sessionKey, final byte[] aggregate) {
@Override
public void remove(final Windowed<Bytes> sessionKey) {
final ConcurrentNavigableMap<Bytes, ConcurrentNavigableMap<Long, byte[]>> keyMap = endTimeMap.get(sessionKey.window().end());
if (keyMap == null) {

This comment has been minimized.

Copy link
@mjsax

mjsax Jul 29, 2019

Member

How could keyMap be null, ie, why would we call remove()for a non-existing session (at least in the DSL)?

Not saying that the fix does not make sense, I am just wondering if we need an additional fix for the DSL?

This comment has been minimized.

Copy link
@mjsax

mjsax Jul 29, 2019

Member

Well. Thinking about it twice, during restore maybe? Tombstone are preserved longer in the changelog, hence, for a deleted session we only have the tombstone. On restore, we would never see and "insert" but only the delete.

Maybe worth double checking in the DSL code anyway.

This comment has been minimized.

Copy link
@ableegoldman

ableegoldman Jul 29, 2019

Author Contributor

I assume a tombstone was being restored, where the corresponding put was already cleaned up from the changelog?

This comment has been minimized.

Copy link
@ableegoldman

ableegoldman Jul 29, 2019

Author Contributor

Answered yourself quicker than me. But ok, I'll look around regardless

This comment has been minimized.

Copy link
@ableegoldman

ableegoldman Jul 30, 2019

Author Contributor

I'm fairly confident the only affected case is on the restore path when the changelog contains a tombstone that outlived its corresponding "put" -- during normal processing, if the delete arrives after the key has already been cleaned up it will be dropped as a late record anyways. Assuming we aren't accidentally sending double tombstones anywhere.

@mjsax

This comment has been minimized.

Copy link
Member

commented Jul 30, 2019

This bug basically make the in-memory store unusable in practice. We should have a proper ticket for tracking. It's not just a HOTFIX.

@jonathanpdx

This comment has been minimized.

Copy link

commented Jul 30, 2019

@ableegoldman

This comment has been minimized.

Copy link
Contributor Author

commented Jul 30, 2019

Thanks for the ticket!

Java8 and Java 11/2.12 passed, Java 11/2.13 failed with known flaky test org.apache.kafka.connect.integration.RebalanceSourceConnectorsIntegrationTest.testStartTwoConnectors

retest this, please

@ableegoldman ableegoldman changed the title HOTFIX: InMemorySessionStore throws NPE removing non-existent key KAFKA-8731: InMemorySessionStore throws NullPointerException on startup Jul 30, 2019

@mjsax
mjsax approved these changes Jul 30, 2019
Copy link
Member

left a comment

Thanks for the quick fix! LGTM.

@bbejeck
Copy link
Contributor

left a comment

Thanks for the fix @ableegoldman LGTM

@mjsax

This comment has been minimized.

Copy link
Member

commented Jul 31, 2019

RebalanceSourceConnectorsIntegrationTest failed for Java11 / 2.12 and Java8.

Java 11 / 2.13 passed.

Retest this please.

@bbejeck bbejeck added the streams label Jul 31, 2019

@bbejeck bbejeck merged commit a028f59 into apache:trunk Jul 31, 2019

3 checks passed

JDK 11 and Scala 2.12 SUCCESS 11728 tests run, 67 skipped, 0 failed.
Details
JDK 11 and Scala 2.13 SUCCESS 11728 tests run, 67 skipped, 0 failed.
Details
JDK 8 and Scala 2.11 SUCCESS 11728 tests run, 67 skipped, 0 failed.
Details
@bbejeck

This comment has been minimized.

Copy link
Contributor

commented Jul 31, 2019

Merged #7132 into trunk

bbejeck added a commit that referenced this pull request Jul 31, 2019
KAFKA-8731: InMemorySessionStore throws NullPointerException on start…
…up (#7132)

Reviewers:  Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.com>
@bbejeck

This comment has been minimized.

Copy link
Contributor

commented Jul 31, 2019

cherry-picked to 2.3

ijuma added a commit to confluentinc/kafka that referenced this pull request Aug 4, 2019
Merge remote-tracking branch 'apache-github/2.3' into ccs-2.3
* apache-github/2.3:
  MINOR: Avoid dividing by zero (apache#7143)
  KAFKA-8731: InMemorySessionStore throws NullPointerException on startup (apache#7132)
  KAFKA-8715; Fix buggy reliance on state timestamp in static member.id generation (apache#7116)
  KAFKA-8678; Fix leave group protocol bug in throttling and error response (apache#7101)
xiowu0 added a commit to linkedin/kafka that referenced this pull request Aug 22, 2019
[LI-CHERRY-PICK] [f86f7e2] KAFKA-8731: InMemorySessionStore throws Nu…
…llPointerException on startup (apache#7132)

TICKET = KAFKA-8731
LI_DESCRIPTION =
EXIT_CRITERIA = HASH [f86f7e2]
ORIGINAL_DESCRIPTION =

Reviewers:  Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.com>
(cherry picked from commit f86f7e2)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
4 participants
You can’t perform that action at this time.