KAFKA-13264: fix inMemoryWindowStore backward fetch not in reversed order#11292
KAFKA-13264: fix inMemoryWindowStore backward fetch not in reversed order#11292guozhangwang merged 4 commits intoapache:trunkfrom
Conversation
| store.put(0, "zero", startTime); | ||
| store.put(1, "one", startTime + 1L); | ||
| store.put(2, "two", startTime + 2L); | ||
| store.put(3, "three", startTime + 2L); |
There was a problem hiding this comment.
add 2 records at the same timestamp to test the forward and backward fetch cases.
| 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(); |
There was a problem hiding this comment.
Before this change, when setting records iterator, we only consider the allKey case, not the forward/backward cases. Fix it.
|
@jeqo @ableegoldman @guozhangwang , please help review this PR. Thank you. |
| final List<StoreType> types = Arrays.asList(StoreType.InMemory, StoreType.RocksDB, StoreType.Timed); | ||
| final List<Boolean> logging = Arrays.asList(true, false); | ||
| final List<Boolean> caching = Arrays.asList(true, false); | ||
| final List<Boolean> forward = Arrays.asList(true, false); |
There was a problem hiding this comment.
Test all kinds of combination.
|
|
||
| private TimeWindowedKStream<String, String> windowedStream; | ||
|
|
||
| public WindowStoreFetchIntegrationTest(final StoreType storeType, final boolean enableLogging, final boolean enableCaching, final boolean forward) { |
There was a problem hiding this comment.
Seems more like a unit test than an integration (though a particularly thorough one😜 )
ableegoldman
left a comment
There was a problem hiding this comment.
LGTM, but can you move the parametrized test out of the integration tests? It's not exactly a clear definition, but we usually consider as integration test the more heavy/long-running tests, eg those that start up an actual KafkaStreams, read/write/verify data, etc
guozhangwang
left a comment
There was a problem hiding this comment.
Thanks for the fix @showuon ! It lgtm. I also feel the same as @ableegoldman that it seems a unit test would suffice, which would take much less time as well :)
| final KeyValue<Windowed<Integer>, String> zero = windowedPair(0, "zero", defaultStartTime); | ||
| final KeyValue<Windowed<Integer>, String> one = windowedPair(1, "one", defaultStartTime + 1); | ||
| final KeyValue<Windowed<Integer>, String> two = windowedPair(2, "two", defaultStartTime + 2); | ||
| final KeyValue<Windowed<Integer>, String> three = windowedPair(3, "three", defaultStartTime + 2); |
There was a problem hiding this comment.
Is this supposed to be defaultStartTime + 3?
There was a problem hiding this comment.
Thanks for your comment, but here, I make it as defaultStartTime + 2 on purpose, to test the case that when window starts time is the same, the forward/backward query API can return the order as expected. I updated the PR description to make it clear. Thank you.
|
@ableegoldman @guozhangwang , thanks for your comments. I agree that it doesn't need to be integration test. I've moved it out from integration test. Thank you. Failed tests are unrelated. |
guozhangwang
left a comment
There was a problem hiding this comment.
Thanks @showuon . I still have a few questions in mind just for my own clarification:
-
Do you know why the original test cases in AbstractWindowBytesStoreTest, like
shouldGetBackwardAllandtestBackwardFetchRangedid not capture this bug? This test class is leveraged by the in-memory stores as well. -
Related to 1), what additional coverage does the new
WindowStoreFetchTestprovides in addition to the above two test cases?
|
@guozhangwang , those are good questions. Let me answer them below:
That's right, those tests also tested in-memory stores, but it didn't test multiple records in the same window cases. Currently, in Window store, we store records in [segments -> [records] ]. For example: key: "a", value: "aa", timestamp: 0 ==> will be in [0, 500] window So, internally, the "a" and "b" will be in the same segment, and "c" in another segments. Before this change, we did have a reverse iterator for segments, but not in "records". So, when doing So, back to the question: why did the original test cases not catch this issue?
I think I've added above. I added an additional record for The behavior works as expected in Hope that's clear. |
Thanks! |
|
Merged to trunk. |
…rder (apache#11292) When introducing backward iterator for WindowStroe in apache#9138, we forgot to make "each segment" in reverse order (i.e. in descendingMap) in InMemoryWindowStore. Fix it and add integration tests for it. Currently, in Window store, we store records in [segments -> [records] ]. For example: window size = 500, input records: key: "a", value: "aa", timestamp: 0 ==> will be in [0, 500] window key: "b", value: "bb", timestamp: 10 ==> will be in [0, 500] window key: "c", value: "cc", timestamp: 510 ==> will be in [500, 1000] window So, internally, the "a" and "b" will be in the same segment, and "c" in another segments. segments: [0 /* window start */, records], [500, records]. And the records for window start 0 will be "a" and "b". the records for window start 500 will be "c". Before this change, we did have a reverse iterator for segments, but not in "records". So, when doing backwardFetchAll, we'll have the records returned in order: "c", "a", "b", which should be "c", "b", "a" obviously. Reviewers: Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Guozhang Wang <wangguoz@gmail.com>
When introducing backward iterator for WindowStroe in #9138, we forgot to make "each segment" in reverse order (i.e. in
descendingMap) inInMemoryWindowStore. Fix it and add integration tests for it.Currently, in Window store, we store records in [segments -> [records] ].
For example:
window size = 500,
input records:
key: "a", value: "aa", timestamp: 0 ==> will be in [0, 500] window
key: "b", value: "bb", timestamp: 10 ==> will be in [0, 500] window
key: "c", value: "cc", timestamp: 510 ==> will be in [500, 1000] window
So, internally, the "a" and "b" will be in the same segment, and "c" in another segments.
segments: [0 /* window start */, records], [500, records].
And the records for window start 0 will be "a" and "b".
the records for window start 500 will be "c".
Before this change, we did have a reverse iterator for segments, but not in "records". So, when doing backwardFetchAll, we'll have the records returned in order: "c", "a", "b", which should be "c", "b", "a" obviously.
So, back to the question: why did the original test cases not catch this issue?
It's because the test input are all in different window start timestamp, which will have different different segments:
I added an additional record for
AbstractWindowBytesStoreTesttest.InWindowStoreFetchTest, we will produce records in timestamp:0, 1, 500, 501, 502, which will be put into window: [0, 500] * 2 and [500, 1000] * 3. And we try to fetch them forward/backward, to see if the results are as expected, i.e.: in reverse order should be 502, 501, 500, 1, 0.The behavior works as expected in RocksDBWindowStore.
Committer Checklist (excluded from commit message)