-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store #4801
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
Changes from all commits
c9efd08
d97ef62
f078efe
111ceb3
16e2a57
9e8d65c
01be095
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
| package org.apache.kafka.streams.state.internals; | ||
|
|
||
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.kafka.common.utils.AbstractIterator; | ||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.common.utils.Utils; | ||
| import org.apache.kafka.streams.KeyValue; | ||
|
|
@@ -441,46 +442,46 @@ private void closeOpenIterators() { | |
| } | ||
| } | ||
|
|
||
| private class RocksDbIterator implements KeyValueIterator<Bytes, byte[]> { | ||
| private class RocksDbIterator extends AbstractIterator<KeyValue<Bytes, byte[]>> implements KeyValueIterator<Bytes, byte[]> { | ||
| private final String storeName; | ||
| private final RocksIterator iter; | ||
|
|
||
| private volatile boolean open = true; | ||
|
|
||
| private KeyValue<Bytes, byte[]> next; | ||
|
|
||
| RocksDbIterator(final String storeName, | ||
| final RocksIterator iter) { | ||
| this.iter = iter; | ||
| this.storeName = storeName; | ||
| } | ||
|
|
||
| byte[] peekRawKey() { | ||
| return iter.key(); | ||
| } | ||
|
|
||
| private KeyValue<Bytes, byte[]> getKeyValue() { | ||
| return new KeyValue<>(new Bytes(iter.key()), iter.value()); | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized boolean hasNext() { | ||
| if (!open) { | ||
| throw new InvalidStateStoreException(String.format("RocksDB store %s has closed", storeName)); | ||
| } | ||
|
|
||
| return iter.isValid(); | ||
| return super.hasNext(); | ||
| } | ||
|
|
||
| /** | ||
| * @throws NoSuchElementException if no next element exist | ||
| */ | ||
| @Override | ||
| public synchronized KeyValue<Bytes, byte[]> next() { | ||
| if (!hasNext()) | ||
| throw new NoSuchElementException(); | ||
| return super.next(); | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure if required but this method was
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ack. |
||
| } | ||
|
|
||
| final KeyValue<Bytes, byte[]> entry = this.getKeyValue(); | ||
| iter.next(); | ||
| return entry; | ||
| @Override | ||
| public KeyValue<Bytes, byte[]> makeNext() { | ||
| if (!iter.isValid()) { | ||
| return allDone(); | ||
| } else { | ||
| next = this.getKeyValue(); | ||
| iter.next(); | ||
| return next; | ||
| } | ||
| } | ||
|
|
||
| private KeyValue<Bytes, byte[]> getKeyValue() { | ||
| return new KeyValue<>(new Bytes(iter.key()), iter.value()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A nit (and paranoid) comment: maybe we can reuse the same
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ack
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. With another look, NM that won't work.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see. Do not bother then :) At lease we are not introduce a regression to make perf worse :)
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am late thus just a meta comment: we hand the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah, that's an excellent point. |
||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -500,7 +501,7 @@ public Bytes peekNextKey() { | |
| if (!hasNext()) { | ||
| throw new NoSuchElementException(); | ||
| } | ||
| return new Bytes(iter.key()); | ||
| return next.key; | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -524,8 +525,17 @@ private class RocksDBRangeIterator extends RocksDbIterator { | |
| } | ||
|
|
||
| @Override | ||
| public synchronized boolean hasNext() { | ||
| return super.hasNext() && comparator.compare(super.peekRawKey(), this.rawToKey) <= 0; | ||
| public KeyValue<Bytes, byte[]> makeNext() { | ||
| final KeyValue<Bytes, byte[]> next = super.makeNext(); | ||
|
|
||
| if (next == null) { | ||
| return allDone(); | ||
| } else { | ||
| if (comparator.compare(next.key.get(), this.rawToKey) <= 0) | ||
| return next; | ||
| else | ||
| return allDone(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,6 +31,7 @@ | |
| import org.junit.Test; | ||
|
|
||
| import java.util.Arrays; | ||
| import java.util.Collections; | ||
| import java.util.NoSuchElementException; | ||
|
|
||
| import static org.junit.Assert.assertEquals; | ||
|
|
@@ -104,6 +105,19 @@ public void shouldIterateOverAllSegments() { | |
| assertFalse(iterator.hasNext()); | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldNotThrowExceptionOnHasNextWhenStoreClosed() { | ||
| iterator = new SegmentIterator(Collections.singletonList(segmentOne).iterator(), | ||
| hasNextCondition, | ||
| Bytes.wrap("a".getBytes()), | ||
| Bytes.wrap("z".getBytes())); | ||
|
|
||
|
|
||
| iterator.currentIterator = segmentOne.all(); | ||
| segmentOne.close(); | ||
| assertFalse(iterator.hasNext()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I guess that comment about verifying that |
||
| } | ||
|
|
||
| @Test | ||
| public void shouldOnlyIterateOverSegmentsInRange() { | ||
| iterator = new SegmentIterator(Arrays.asList(segmentOne, segmentTwo).iterator(), | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
Having this check here has got me to thinking more about this issue.
Without this guard condition, we have some failing unit tests.
In both the
RocksDBIteratorand theAbstractIteratorall calls tonext()make a call tohasNext()first before returning the next object. I'm not sure about changing the semantics where we return fromnext()without callinghasNext()first (which if we end up keeping those semantics, leaves us in the same position as before extendingAbstractIterator).I guess the question is, do we want to continue to throw an exception when
hasNext()is called (when the store is closed) or simply returnfalse?I could be overthinking this, but I'm not entirely comfortable with returning a value from
next()after closing the store. I feel like that creates more corner cases for potential errors or unexpected behavior.WDYT?
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.
@bbejeck That is a good question!
Originally I thought it is okay to always calling
hasNextinsidenext(), as long as we make surehasNextimplementation is idempotent, i.e. calling it multiple times beforenext()does not have side effect is sufficient. But by making it idempotent we could have the corner case you mentioned. For example:Without this check, at
t3we would still return thenextfield.