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
[9/N][Emit final] Emit final for session window aggregations #12204
Changes from 1 commit
d40ac90
fadd900
16c4ebd
5ef28a3
e37249a
fda662f
3942fc6
66c7f35
0a0f35c
484de9d
da1d91e
d48119f
578526a
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 |
---|---|---|
|
@@ -39,7 +39,7 @@ | |
*/ | ||
public class RocksDBTimeOrderedSessionSegmentedBytesStore extends AbstractRocksDBTimeOrderedSegmentedBytesStore { | ||
|
||
private class SessionKeySchemaIndexToBaseStoreIterator extends IndexToBaseStoreIterator { | ||
private class SessionKeySchemaIndexToBaseStoreIterator extends IndexToBaseStoreIterator { | ||
SessionKeySchemaIndexToBaseStoreIterator(final KeyValueIterator<Bytes, byte[]> indexIterator) { | ||
super(indexIterator); | ||
} | ||
|
@@ -76,10 +76,10 @@ public KeyValueIterator<Bytes, byte[]> fetchSessions(final long earliestSessionE | |
final long latestSessionEndTime) { | ||
final List<KeyValueSegment> searchSpace = segments.segments(earliestSessionEndTime, latestSessionEndTime, true); | ||
|
||
// here we use both as lower range since we the search boundaries are both on the session time end; | ||
// hence effectively our search range is [0 earliestSE] - [0, latestSE] | ||
// here we want [0, latestSE, FF] as the upper bound to cover any possible keys, | ||
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. This is the other minor bug I detected in the latest commit. |
||
// but since we can only get upper bound based on timestamps, we use a slight larger upper bound as [0, latestSE+1] | ||
final Bytes binaryFrom = baseKeySchema.lowerRangeFixedSize(null, earliestSessionEndTime); | ||
final Bytes binaryTo = baseKeySchema.lowerRangeFixedSize(null, latestSessionEndTime); | ||
final Bytes binaryTo = baseKeySchema.lowerRangeFixedSize(null, latestSessionEndTime + 1); | ||
|
||
return new SegmentIterator<>( | ||
searchSpace.iterator(), | ||
|
@@ -90,9 +90,9 @@ public KeyValueIterator<Bytes, byte[]> fetchSessions(final long earliestSessionE | |
final Windowed<Bytes> windowedKey = TimeFirstSessionKeySchema.from(bytes); | ||
final long endTime = windowedKey.window().end(); | ||
|
||
if (endTime <= latestSessionEndTime && endTime >= earliestSessionEndTime) | ||
if (endTime <= latestSessionEndTime && endTime >= earliestSessionEndTime) { | ||
return true; | ||
|
||
} | ||
iterator.next(); | ||
} | ||
return false; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -54,24 +54,18 @@ private SessionStore<Bytes, byte[]> maybeWrapCaching(final SessionStore<Bytes, b | |
return inner; | ||
} | ||
|
||
if (!inner.persistent()) { | ||
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. Here's the other change I made in order to work around the current tricky situation, since in-memory stores are always "time ordered" as well, we stripe the caching if the inner store is not persistent. cc @mjsax 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. If we strip the caching, this applies to eager emitting, right? -- So it would be a behavioral change? Do we want to piggy-back such a change into this KIP? Sounds "risky"? 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. Sounds good, I will move this logic into the earlier stage during the topology building phase for the moment. Also cc @lihaosky who would do similar things for sliding windows. |
||
return inner; | ||
} | ||
|
||
// do not enable cache if the underlying store is time ordered | ||
if (isTimeOrderedStore(inner)) { | ||
if (storeSupplier instanceof RocksDbTimeOrderedSessionBytesStoreSupplier) { | ||
return inner; | ||
} | ||
|
||
return new CachingSessionStore(inner, storeSupplier.segmentIntervalMs()); | ||
} | ||
|
||
private boolean isTimeOrderedStore(final StateStore stateStore) { | ||
if (stateStore instanceof RocksDBTimeOrderedWindowStore) { | ||
return true; | ||
} | ||
if (stateStore instanceof WrappedStateStore) { | ||
return isTimeOrderedStore(((WrappedStateStore) stateStore).wrapped()); | ||
} | ||
return false; | ||
} | ||
|
||
private SessionStore<Bytes, byte[]> maybeWrapLogging(final SessionStore<Bytes, byte[]> inner) { | ||
if (!enableLogging) { | ||
return inner; | ||
|
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.
This is one minor bug I detected in the latest commit.