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-13264: fix inMemoryWindowStore backward fetch not in reversed order #11292

Merged
merged 4 commits into from Sep 13, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -499,10 +499,14 @@ Iterator<Map.Entry<Bytes, byte[]>> setRecordIterator() {
final Map.Entry<Long, ConcurrentNavigableMap<Bytes, byte[]>> currentSegment = segmentIterator.next();
currentTime = currentSegment.getKey();

if (allKeys) {
return currentSegment.getValue().entrySet().iterator();
final ConcurrentNavigableMap<Bytes, byte[]> subMap = allKeys ?
currentSegment.getValue() :
currentSegment.getValue().subMap(keyFrom, true, keyTo, true);

if (forward) {
return subMap.entrySet().iterator();
} else {
return currentSegment.getValue().subMap(keyFrom, true, keyTo, true).entrySet().iterator();
return subMap.descendingMap().entrySet().iterator();
Comment on lines -502 to +509
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Before this change, when setting records iterator, we only consider the allKey case, not the forward/backward cases. Fix it.

}
}

Expand Down