KAFKA-5172: Fix fetchPrevious to find the correct session.#2972
KAFKA-5172: Fix fetchPrevious to find the correct session.#2972ghost wants to merge 4 commits intoapache:trunkfrom kyle-winkelman:CachingSessionStore-fetchPrevious
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): |
|
@dguy could you take a look? |
dguy
left a comment
There was a problem hiding this comment.
@KyleWinkelman thanks for the PR. Can you please add a test to CachingSessionStoreTest that fails without this change?
|
@dguy I have added a test case that fails without this change. |
|
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
left a comment
There was a problem hiding this comment.
Thanks for adding the test @KyleWinkelman left a couple of comments
| */ | ||
| package org.apache.kafka.streams.state.internals; | ||
|
|
||
| import static org.apache.kafka.streams.state.internals.RocksDBSessionStoreTest.toList; |
There was a problem hiding this comment.
can you re-order the input back to what they where before?
| @Test | ||
| public void shouldPutAndMaybeForward() throws Exception { | ||
| final Windowed<String> a1 = new Windowed<>("a", new SessionWindow(0, 0)); | ||
| final Windowed<String> a2 = new Windowed<>("a", new SessionWindow(0, 1)); |
There was a problem hiding this comment.
I don't think we need to add the extra class just for this case. We could do something like:
final List<Change<Long>> values = new ArrayList<>();
cachingStore.setFlushListener(new CacheFlushListener<Windowed<String>, Long>() {
@Override
public void apply(final Windowed<String> key, final Long newValue, final Long oldValue) {
values.add(new Change<>(newValue, oldValue));
}
});
...
assertThat(values, equalTo(Arrays.asList(new Change<>(1L, null), new Change<>(null, 1L), new Change<>(2L, null))));
| assertFalse(results.hasNext()); | ||
| } | ||
|
|
||
| @Test |
There was a problem hiding this comment.
maybe something like: shouldForwardChangedValuesDuringFlush ?
| import org.apache.kafka.streams.kstream.internals.CacheFlushListener; | ||
| import org.apache.kafka.streams.kstream.internals.Change; | ||
|
|
||
| public class MockCacheFlushListener<K, V> implements CacheFlushListener<K, V> { |
There was a problem hiding this comment.
See comment above, i don't think we need this class
|
I made the recommended changes and altered the contents of the test a little so we could see a case where the new and old values are set. |
|
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 Do you think we should prevent the Bytes.wrap(serdes.rawKey(key.key())) from being called twice (once in putAndMaybeForward and once in fetchPrevious)? I don't know how costly this is but it could save a little bit of processing. I think we could call it once and change the fetchPrevious to accept Bytes and Window. |
|
@KyleWinkelman yes that makes sense. Thanks! |
|
Ok made the changes. Anything else required of me for this PR? |
|
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): |
|
retest this please |
|
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): |
|
retest this please |
|
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): |
guozhangwang
left a comment
There was a problem hiding this comment.
LGTM! Merged to trunk.
…for flushing / getter (#6161) #2972 tried to fix a bug about flushing operation, but it was not complete, since findSessions(key, earliestEnd, latestStart) does not guarantee to only return a single entry since its semantics are to return any sessions whose end > earliestEnd and whose start < latestStart. I've tried various ways to fix it completely and I ended up having to add a single-point query to the public ReadOnlySessionStore API for the exact needed semantics. It is used for flushing to read the old values (otherwise the wrong old values will be sent downstreams, hence it is a correctness issue) and also for getting the value for value-getters (it is for perf only).
…for flushing / getter (apache#6161) apache#2972 tried to fix a bug about flushing operation, but it was not complete, since findSessions(key, earliestEnd, latestStart) does not guarantee to only return a single entry since its semantics are to return any sessions whose end > earliestEnd and whose start < latestStart. I've tried various ways to fix it completely and I ended up having to add a single-point query to the public ReadOnlySessionStore API for the exact needed semantics. It is used for flushing to read the old values (otherwise the wrong old values will be sent downstreams, hence it is a correctness issue) and also for getting the value for value-getters (it is for perf only).
Change fetchPrevious to use findSessions with the proper key and timestamps rather than using fetch.