-
Notifications
You must be signed in to change notification settings - Fork 5
Mongo window range queries #224
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
base: main
Are you sure you want to change the base?
Changes from all commits
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 |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
| import static com.mongodb.MongoClientSettings.getDefaultCodecRegistry; | ||
| import static dev.responsive.kafka.internal.db.partitioning.SegmentPartitioner.UNINITIALIZED_STREAM_TIME; | ||
| import static dev.responsive.kafka.internal.stores.ResponsiveStoreRegistration.NO_COMMITTED_OFFSET; | ||
| import static dev.responsive.kafka.internal.stores.SegmentedOperations.MIN_KEY; | ||
| import static org.bson.codecs.configuration.CodecRegistries.fromProviders; | ||
| import static org.bson.codecs.configuration.CodecRegistries.fromRegistries; | ||
|
|
||
|
|
@@ -52,6 +53,7 @@ | |
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.streams.KeyValue; | ||
| import org.apache.kafka.streams.errors.TaskMigratedException; | ||
| import org.apache.kafka.streams.kstream.Windowed; | ||
| import org.apache.kafka.streams.state.KeyValueIterator; | ||
| import org.bson.codecs.configuration.CodecProvider; | ||
| import org.bson.codecs.configuration.CodecRegistry; | ||
|
|
@@ -457,13 +459,37 @@ public KeyValueIterator<WindowedKey, byte[]> backFetch( | |
| @Override | ||
| public KeyValueIterator<WindowedKey, byte[]> fetchRange( | ||
| final int kafkaPartition, | ||
| final Bytes fromKey, | ||
| final Bytes toKey, | ||
| final Bytes keyFrom, | ||
| final Bytes keyTo, | ||
| final long timeFrom, | ||
| final long timeTo | ||
| ) { | ||
| throw new UnsupportedOperationException("fetchRange not yet supported for Mongo backends"); | ||
| final List<KeyValueIterator<WindowedKey, byte[]>> segmentIterators = new LinkedList<>(); | ||
| final var partitionSegments = kafkaPartitionToSegments.get(kafkaPartition); | ||
|
|
||
| for (final var segment : partitioner.range(kafkaPartition, timeFrom, timeTo)) { | ||
| final var segmentWindows = partitionSegments.segmentWindows.get(segment); | ||
|
|
||
| // Since we use a flat keyspace by concatenating the timestamp with the data key and have | ||
| // variable length data keys, it's impossible to request only valid data that's within | ||
|
Contributor
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. question to clarify my understanding - why does the variability of the length of the data key matter? even if we knew all the keys were the same size, this filter would still pick up too much data right? if the key is first, you'd still pick up a bunch of time windows you don't want. And if the time is first you'd still pick up a bunch of keys you don't want. |
||
| // the given bounds. Instead issue a broad request from the valid bounds and then post-filter | ||
| final var lowerBound = compositeKey(keyFrom, timeFrom); | ||
| final var upperBound = compositeKey(keyTo, timeTo); | ||
|
|
||
| final FindIterable<WindowDoc> fetchResults = segmentWindows.find( | ||
| Filters.and( | ||
|
Contributor
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. I'm wondering if we should include these fields outside of the key and filter on them server-side. This has the potential to jack up our data transfer costs.
Contributor
Author
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. Not sure I understand the question -- what are "these fields"? And what do you mean by including them "outside the key" -- are you talking about the composite key, and moving the timestamp to a separate field rather than a composite? I don't see how that would affect our data transfer costs for range queries, as in all cases where we return the key we would need to return the timestamp as well. I'm wondering if there might be a case for point lookups, though. Since AFAICT there's no way to return a doc without sending the key/ID, whereas if the timestamp was an additional field then maybe it's possible to filter that out. Is that what you're getting at? Is it even true that there's no way to do a point lookup without returning the key/ID in the response? I'm assuming that it's possible to filter out other fields besides the key, but even that I don't know for sure 🤔 |
||
| Filters.gte(WindowDoc.ID, lowerBound), | ||
| Filters.lte(WindowDoc.ID, upperBound)) | ||
| ); | ||
|
|
||
|
|
||
| segmentIterators.add( | ||
| Iterators.filterKv( | ||
| Iterators.kv(fetchResults.iterator(), MongoWindowedTable::windowFromDoc), | ||
| kv -> filterFetchRange(kv, timeFrom, timeTo, keyFrom, keyTo, timestampFirstOrder)) | ||
| ); | ||
| } | ||
| return Iterators.wrapped(segmentIterators); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -483,7 +509,35 @@ public KeyValueIterator<WindowedKey, byte[]> fetchAll( | |
| final long timeFrom, | ||
| final long timeTo | ||
| ) { | ||
| throw new UnsupportedOperationException("fetchAll not yet supported for Mongo backends"); | ||
| if (!timestampFirstOrder) { | ||
| throw new UnsupportedOperationException("Range queries such as fetchAll require stores to be " | ||
|
Contributor
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 because mongo doesn't let you scan the whole table?
Contributor
Author
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. I guess this was a "prevent users from shooting themselves in the foot" moment -- it's possible, just really inefficient since we have to scan everything in between (see comment about variable length keys). Of course if we're going to go the route of scanning everything in order to enable partition scaling, then this constraint doesn't make any sense. I'll remove it |
||
| + "configured with timestamp-first order"); | ||
| } | ||
|
|
||
| final List<KeyValueIterator<WindowedKey, byte[]>> segmentIterators = new LinkedList<>(); | ||
| final var partitionSegments = kafkaPartitionToSegments.get(kafkaPartition); | ||
|
|
||
| for (final var segment : partitioner.range(kafkaPartition, timeFrom, timeTo)) { | ||
| final var segmentWindows = partitionSegments.segmentWindows.get(segment); | ||
|
|
||
| // To avoid scanning the entire segment, we use the bytewise "minimum key" to start the scan | ||
| // at the lower time bound. Since there's no corresponding "maximum key" given the variable | ||
| // length keys, we have to set the upper bound at timeTo + 1, while using strict comparison | ||
| // (ie #lt rather than #lte) to exclude said upper bound | ||
| final var lowerBound = compositeKey(MIN_KEY, timeFrom); | ||
| final var upperBound = compositeKey(MIN_KEY, timeTo + 1); | ||
|
|
||
| final FindIterable<WindowDoc> fetchResults = segmentWindows.find( | ||
| Filters.and( | ||
| Filters.gte(WindowDoc.ID, lowerBound), | ||
| Filters.lt(WindowDoc.ID, upperBound)) | ||
| ); | ||
|
|
||
| segmentIterators.add( | ||
| Iterators.kv(fetchResults.iterator(), MongoWindowedTable::windowFromDoc) | ||
| ); | ||
| } | ||
| return Iterators.wrapped(segmentIterators); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -495,6 +549,34 @@ public KeyValueIterator<WindowedKey, byte[]> backFetchAll( | |
| throw new UnsupportedOperationException("backFetchAll not yet supported for MongoDB backends"); | ||
| } | ||
|
|
||
| @Override | ||
| public KeyValueIterator<WindowedKey, byte[]> all( | ||
| final int kafkaPartition, | ||
| final long streamTime | ||
|
Contributor
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. don't we need to filter on this time? |
||
| ) { | ||
| final List<KeyValueIterator<WindowedKey, byte[]>> segmentIterators = new LinkedList<>(); | ||
| final var partitionSegments = kafkaPartitionToSegments.get(kafkaPartition); | ||
|
|
||
| for (final var segment : partitioner.activeSegments(kafkaPartition, streamTime)) { | ||
| final var segmentWindows = partitionSegments.segmentWindows.get(segment); | ||
|
|
||
| final FindIterable<WindowDoc> fetchResults = segmentWindows.find(); | ||
|
|
||
| segmentIterators.add( | ||
| Iterators.kv(fetchResults.iterator(), MongoWindowedTable::windowFromDoc) | ||
| ); | ||
| } | ||
| return Iterators.wrapped(segmentIterators); | ||
| } | ||
|
|
||
| @Override | ||
| public KeyValueIterator<WindowedKey, byte[]> backAll( | ||
| final int kafkaPartition, | ||
| final long streamTime | ||
| ) { | ||
| throw new UnsupportedOperationException("backAll not yet supported for MongoDB backends"); | ||
| } | ||
|
|
||
| public BasicDBObject compositeKey(final WindowedKey windowedKey) { | ||
| return compositeKey(windowedKey.key, windowedKey.windowStartMs); | ||
| } | ||
|
|
@@ -511,5 +593,22 @@ private static KeyValue<WindowedKey, byte[]> windowFromDoc(final WindowDoc windo | |
| return new KeyValue<>(WindowDoc.windowedKey(windowDoc.id), windowDoc.value); | ||
| } | ||
|
|
||
| private static boolean filterFetchRange( | ||
| final WindowedKey windowedKey, | ||
| final long timeFrom, | ||
| final long timeTo, | ||
| final Bytes keyFrom, | ||
| final Bytes keyTo, | ||
| final boolean timestampFirstOrder | ||
| ) { | ||
| // If we use timestamp-first order, then the upper/lower bounds guarantee the timestamps are | ||
| // valid, so therefore we only need to filter out the invalid keys (and vice versa) | ||
| if (timestampFirstOrder) { | ||
| return windowedKey.key.compareTo(keyFrom) > 0 && windowedKey.key.compareTo(keyTo) < 0; | ||
| } else { | ||
| return windowedKey.windowStartMs > timeFrom && windowedKey.windowStartMs < timeTo; | ||
| } | ||
| } | ||
|
|
||
| } | ||
|
|
||
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.
question about the state store api in general - what are these cross-key APIs actually used for? All the kafka streams processors are scoped to one key afaik.