[ https://issues.apache.org/jira/browse/KAFKA-5172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15996038#comment-15996038 ]
ASF GitHub Bot commented on KAFKA-5172: --------------------------------------- GitHub user KyleWinkelman opened a pull request: https://github.com/apache/kafka/pull/2972 [KAFKA-5172] Fix fetchPrevious to find the correct session. Change fetchPrevious to use findSessions with the proper key and timestamps rather than using fetch. You can merge this pull request into a Git repository by running: $ git pull https://github.com/KyleWinkelman/kafka CachingSessionStore-fetchPrevious Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/2972.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2972 ---- commit c09a1ace192877ef9f633e14288b680f374b3792 Author: Kyle Winkelman <kyle.winkel...@optum.com> Date: 2017-05-04T02:03:50Z Fix fetchPrevious to find the correct session. ---- > CachingSessionStore doesn't fetchPrevious correctly. > ---------------------------------------------------- > > Key: KAFKA-5172 > URL: https://issues.apache.org/jira/browse/KAFKA-5172 > Project: Kafka > Issue Type: Bug > Components: streams > Reporter: Kyle Winkelman > > When using KStreamSessionWindowAggregate by calling > KGroupedStream#aggregate() a CachingSessionStore is created. > This causes the following chain of method calls when a new record that > requires removing others from the store appear: > KStreamSessionWindowAggregate > CachingSessionStore.remove(Windowed<K>) > CachingSessionStore.put(Windowed<K>, V) > ThreadCache.put(String, Bytes *containing Windowed<K> info*, LRUCacheEntry) > ThreadCache.maybeEvict(String) > NamedCache.evict() > NamedCache.flush(LRUNode *containing Bytes and LRUCacheEntry from > ThreadCache#put*) > DirtyEntryFlushListener *defined in CachingSessionStore line 80* > .apply(ThreadCache.DirtyEntry *containing Bytes and LRUCacheEntry from > ThreadCache#put*) > CachingSessionStore.putAndMaybeForward(ThreadCache.DirtyEntry *containing > Bytes and LRUCacheEntry from ThreadCache#put*, InternalProcessorContext) > CachingSessionStore.fetchPrevious(Bytes *containing Windowed<K> info*) > RocksDBSessionStore.fetch(Bytes *containing Windowed<K> info*) > RocksDBSessionStore.findSessions *on line 48* (Bytes *containing Windowed<K> > info*, 0, Long.MAX_VALUE) > MeteredSegmentedByteStore.fetch(Bytes *containing Windowed<K> info*, 0, > Long.MAX_VALUE) > ChangeLoggingSegmentedByteStore.fetch(Bytes *containing Windowed<K> info*, 0, > Long.MAX_VALUE) > RocksDBSegmentedBytesStore.fetch(Bytes *containing Windowed<K> info*, 0, > Long.MAX_VALUE) > SessionKeySchema.lower/upperRange(Bytes *containing Windowed<K> info*, Long) > ****** in this method the already Windowed<K> gets Windowed again ***** > The point of showing all this is to point out that the windowed gets windowed > and because it passes the 0, Long.MAX_VALUE it searches for a strange key and > searches all times for it. I think the fetchPrevious method of > CachingSessionStore should be changed to call the > byteStores.findSessions(Bytes.wrap(serdes.rawKey(key.key())), > key.window().start(), key.window().end()). -- This message was sent by Atlassian JIRA (v6.3.15#6346)