Thanks, John! Make sense to reconsider the current approach. I was heading in a similar direction while drafting the implementation. Metered, Caching, and other layers will also have to get duplicated to build up new methods in `Stores` factory, and class casting issues would appear on stores created by DSL.
I will draft a proposal with new methods (move methods from proposed interfaces to existing ones) with default implementation in a KIP update and wait for Matthias to chime in to validate this approach. Jorge. On Sat, Jun 27, 2020 at 4:01 PM John Roesler <vvcep...@apache.org> wrote: > Hi Jorge, > > Sorry for my silence, I've been absorbed with the 2.6 and 2.5.1 releases. > > The idea to separate the new methods into "mixin" interfaces seems > like a good one, but as we've discovered in KIP-614, it doesn't work > out that way in practice. The problem is that the store implementations > are just the base layer that get composed with other layers in Streams > before they can be accessed in the DSL. This is extremely subtle, so > I'm going to put everyone to sleep with a detailed explanation: > > For example, this is the mechanism by which all KeyValueStore > implementations get added to Streams: > org.apache.kafka.streams.state.internals.KeyValueStoreBuilder#build > return new MeteredKeyValueStore<>( > maybeWrapCaching(maybeWrapLogging(storeSupplier.get())), > storeSupplier.metricsScope(), > time, > keySerde, > valueSerde > ); > > In the DSL, the store that a processor gets from the context would be > the result of this composition. So even if the storeSupplier.get() returns > a store that implements the "reverse" interface, when you try to use it > from a processor like: > org.apache.kafka.streams.kstream.ValueTransformerWithKey#init > ReadOnlyBackwardWindowStore<K, V> store = > (ReadOnlyBackwardWindowStore<K, V>) context.getStateStore(..) > > You'd just get a ClassCastException because it's actually a > MeteredKeyValueStore, which doesn't implement > ReadOnlyBackwardWindowStore. > > The only way to make this work would be to make the Metered, > Caching, and Logging layers also implement the new interfaces, > but this effectively forces implementations to also implement > the interface. Otherwise, the intermediate layers would have to > cast the store in each method, like this: > MeteredWindowStore#backwardFetch { > ((ReadOnlyBackwardWindowStore<K, V>) innerStore).backwardFetch(..) > } > > And then if the implementation doesn't "opt in" by implementing > the interface, you'd get a ClassCastException, not when you get the > store, but when you try to use it. > > The fact that we get ClassCastExceptions no matter which way we > turn here indicates that we're really not getting any benefit from the > type system, which makes the extra interfaces seem not worth all the > code involved. > > Where we landed in KIP-614 is that, unless we want to completely > revamp the way that StateStores work in the DSL, you might as > well just add the new methods to the existing interfaces. To prevent > compilation errors, we can add default implementations that throw > UnsupportedOperationException. If a store doesn't opt in by > implementing the methods, you'd get an UnsupportedOperationException, > which seems no worse, and maybe better, than the ClassCastException > you'd get if we go with the "mixin interface" approach. > > A quick note: This entire discussion focuses on the DSL. If you're just > using the Processor API by directly adding the a custom store to the > Topology: > org.apache.kafka.streams.Topology#addStateStore > and then retrieving it in the processor via: > org.apache.kafka.streams.processor.ProcessorContext#getStateStore > in > org.apache.kafka.streams.processor.Processor#init > > Then, you can both register and retrieve _any_ StateStore implementation. > There's no need to use KeyValueStore or any other built-in interface. > In other words, KeyValueStore and company are only part of the DSL, > not the PAPI. So, discussions about the build-in store interfaces are only > really relevant in the context of the DSL, Transformers, and Materialized. > > So, in conclusion, I'd really recommend just adding any new methods to > the existing store interfaces. We might be able to revamp the API in the > future to support mixins, but it's a much larger scope of work than this > KIP. > A more minor comment is that we don't need to add Deprecated variants > of new methods. > > Thanks again, and once again, I'm sorry I tuned out and didn't offer this > feedback before you revised the KIP. > -John > > > > > On Mon, Jun 22, 2020, at 06:11, Jorge Esteban Quilcate Otoya wrote: > > Hi everyone, > > > > I've updated the KIP, applying Matthias' feedback regarding interface > > hierarchy. > > > > Also, following the last email, I think we can consider reverse > operations > > on KeyValue range as well, as implementation supports lexicographic > order. > > > > I considered different naming between Key-based ranges and Time-based > > ranges, and mitigate confusion when fetching keys and time ranges as > > WindowStore does: > > > > Key-based ranges: reverseRange(), reverseAll() > > Time-based ranges: backwardFetch() > > > > Then, key-based changes apply to KeyValueStore, and time-based changes to > > Window and Session stores. > > > > Let me know if you have any questions. > > > > Thanks, > > Jorge. > > > > > > On Tue, Jun 16, 2020 at 12:47 AM Jorge Esteban Quilcate Otoya < > > quilcate.jo...@gmail.com> wrote: > > > > > Hi everyone, sorry for the late reply. > > > > > > Thanks Matthias for your feedback. I think it makes sense to reconsider > > > the current design based on your input. > > > > > > After digging deeper into the current implementation, I'd like to > bring my > > > current understanding to be double-checked as it might be redefining > the > > > KIP's scope: > > > > > > 1. There are 2 ranges been exposed by different stores: > > > > > > a. Key Range > > > b. Timestamp Range > > > > > > So far, we have discussed covering both. > > > > > > 2. Key Range functions do not provide ordering guarantees by design: > > > > > > ```ReadOnlyKeyValueStore.java > > > /** > > > * Get an iterator over a given range of keys. This iterator must > be > > > closed after use. > > > * The returned iterator must be safe from {@link > > > java.util.ConcurrentModificationException}s > > > * and must not return null values. No ordering guarantees are > > > provided. > > > * ... > > > */ > > > KeyValueIterator<K, V> range(K from, K to); > > > ``` > > > > > > Therefore, I'd propose removing Key range operations from the scope. > > > > > > 3. Timestamp Range operations happen at the SegmentsStore level > (internal) > > > API > > > > > > AFAICT, Segments wrappers handle all Timestamp ranges queries. > > > > > > I'd propose extending `Segments#segments(timeFrom, timeTo, backwards)` > > > with a flag for backwards operations. > > > > > > As segments returned will be processed backwards, I'm not extending > > > KeyValueStores to query each segment backwards as previous point 2. > > > > > > 4. Extend WindowStores implementations with a new > > > WindowBackwardStore/ReadOnlyBackwardStore: > > > > > > ```java > > > public interface ReadOnlyWindowBackwardStore<K, V> { > > > WindowStoreIterator<V> backwardFetch(K key, Instant from, Instant > to) > > > throws IllegalArgumentException; > > > > > > KeyValueIterator<Windowed<K>, V> backwardFetch(K from, K to, > Instant > > > fromTime, Instant toTime) > > > throws IllegalArgumentException; > > > > > > KeyValueIterator<Windowed<K>, V> backwardFetchAll(Instant from, > > > Instant to) throws IllegalArgumentException; > > > ``` > > > > > > 5. SessionStore is a bit different as it has fetch/find sessions spread > > > between SessionStore and ReadOnlySessionStore. > > > > > > I'd propose a new interface `SessionBackwardStore` to expose backward > find > > > operations: > > > > > > ```java > > > public interface SessionBackwardStore<K, AGG> { > > > KeyValueIterator<Windowed<K>, AGG> backwardFindSessions(final K > key, > > > final long earliestSessionEndTime, final long latestSessionStartTime); > > > > > > KeyValueIterator<Windowed<K>, AGG> backwardFindSessions(final K > > > keyFrom, final K keyTo, final long earliestSessionEndTime, final long > > > latestSessionStartTime); > > > } > > > ``` > > > > > > If this understanding is correct I'll proceed to update the KIP based > on > > > this. > > > > > > Looking forward to your feedback. > > > > > > Thanks, > > > Jorge. > > > > > > On Fri, May 29, 2020 at 3:32 AM Matthias J. Sax <mj...@apache.org> > wrote: > > > > > >> Hey, > > >> > > >> Sorry that I am late to the game. I am not 100% convinced about the > > >> current proposal. Using a new config as feature flag seems to be > rather > > >> "nasty" to me, and flipping from/to is a little bit too fancy for my > > >> personal taste. > > >> > > >> I agree, that the original proposal using a "ReadDirection" enum is > not > > >> ideal either. > > >> > > >> Thus, I would like to put out a new idea: We could add a new interface > > >> that offers new methods that return revers iterators. > > >> > > >> The KIP already proposes to add `reverseAll()` and it seems backward > > >> incompatible to just add this method to `ReadOnlyKeyValueStore` and > > >> `ReadOnlyWindowStore`. I don't think we could provide a useful default > > >> implementation for custom stores and thus either break compatibility > or > > >> need add a default that just throws an exception. Neither seems to be > a > > >> good option. > > >> > > >> Using a new interface avoid this issue and allows users implementing > > >> custom stores to opt-in by adding the interface to their stores. > > >> Furthermore, we don't need any config. In the end, we encapsulte the > > >> change into the store, and our runtime is agnostic to it (as it should > > >> be). > > >> > > >> The hierarchy becomes a little complex (but uses would not really see > > >> the complexity): > > >> > > >> // exsiting > > >> ReadOnlyKeyValueStore > > >> KeyValueStore extend StateStore, ReadOnlyKeyValueStore > > >> > > >> > > >> // helper interface; users don't care > > >> // need similar ones for other stores > > >> ReverseReadOnlyKeyValueStore { > > >> KeyValueIterator<K, V> reverseRange(K from, K to); > > >> KeyValueIterator<K, V> reverseAll(); > > >> } > > >> > > >> > > >> // two new user facing interfaces for kv-store > > >> // need similar ones for other stores > > >> ReadOnlyKeyValueStoreWithReverseIterators extends > ReadOnlyKeyValueStore, > > >> ReverseReadOnlyKeyValueStore > > >> > > >> KeyValueStoreWithReverseIterators extends KeyValueStore, > > >> ReverseReadOnlyKeyValueStore > > >> > > >> > > >> // updated (also internal) > > >> // also need to update other built-in stores > > >> RocksDB implements KeyValueStoreWithReverseIterators, BulkLoadingStore > > >> > > >> > > >> In the end, user would only care about the two (for kv-case) new > > >> interface that offer revers iterator (read only and regular) and can > > >> cast stores accordingly in their Processors/Transformers or via IQ. > > >> > > >> > > >> Btw: if we add revers iterator for KeyValue and Window store, should > we > > >> do the same for Session store? > > >> > > >> > > >> > > >> This might be more code to write, but I believe it provides the better > > >> user experience. Thoughts? > > >> > > >> > > >> > > >> -Matthias > > >> > > >> > > >> > > >> > > >> On 5/26/20 8:47 PM, John Roesler wrote: > > >> > Sorry for my silence, Jorge, > > >> > > > >> > I've just taken another look, and I'm personally happy with the KIP. > > >> > > > >> > Thanks, > > >> > -John > > >> > > > >> > On Tue, May 26, 2020, at 16:17, Jorge Esteban Quilcate Otoya wrote: > > >> >> If no additional comments, I will proceed to start the a vote > thread. > > >> >> > > >> >> Thanks a lot for your feedback! > > >> >> > > >> >> On Fri, May 22, 2020 at 9:25 AM Jorge Esteban Quilcate Otoya < > > >> >> quilcate.jo...@gmail.com> wrote: > > >> >> > > >> >>> Thanks Sophie. I like the `reverseAll()` idea. > > >> >>> > > >> >>> I updated the KIP with your feedback. > > >> >>> > > >> >>> > > >> >>> > > >> >>> On Fri, May 22, 2020 at 4:22 AM Sophie Blee-Goldman < > > >> sop...@confluent.io> > > >> >>> wrote: > > >> >>> > > >> >>>> Hm, the case of `all()` does seem to present a dilemma in the > case of > > >> >>>> variable-length keys. > > >> >>>> > > >> >>>> In the case of fixed-length keys, you can just compute the keys > that > > >> >>>> correspond > > >> >>>> to the maximum and minimum serialized bytes, then perform a > `range()` > > >> >>>> query > > >> >>>> instead of an `all()`. If your keys don't have a well-defined > > >> ordering > > >> >>>> such > > >> >>>> that > > >> >>>> you can't determine the MAX_KEY, then you probably don't care > about > > >> the > > >> >>>> iterator order anyway. > > >> >>>> > > >> >>>> But with variable-length keys, there is no MAX_KEY. If all your > > >> keys were > > >> >>>> just > > >> >>>> of the form 'a', 'aa', 'aaaaa', 'aaaaaaa' then in fact the only > way > > >> to > > >> >>>> figure out the > > >> >>>> maximum key in the store is by using `all()` -- and without a > reverse > > >> >>>> iterator, you're > > >> >>>> doomed to iterate through every single key just to answer that > simple > > >> >>>> question. > > >> >>>> > > >> >>>> That said, I still think determining the iterator order based on > the > > >> >>>> to/from bytes > > >> >>>> makes a lot of intuitive sense and gives the API a nice symmetry. > > >> What if > > >> >>>> we > > >> >>>> solved the `all()` problem by just giving `all()` a reverse form > to > > >> >>>> complement it? > > >> >>>> Ie we would have `all()` and `reverseAll()`, or something to that > > >> effect. > > >> >>>> > > >> >>>> On Thu, May 21, 2020 at 3:41 PM Jorge Esteban Quilcate Otoya < > > >> >>>> quilcate.jo...@gmail.com> wrote: > > >> >>>> > > >> >>>>> Thanks John. > > >> >>>>> > > >> >>>>> Agree. I like the first approach as well, with StreamsConfig > flag > > >> >>>> passing > > >> >>>>> by via ProcessorContext. > > >> >>>>> > > >> >>>>> Another positive effect with "reverse parameters" is that in the > > >> case of > > >> >>>>> `fetch(keyFrom, keyTo, timeFrom, timeTo)` users can decide > _which_ > > >> pair > > >> >>>> to > > >> >>>>> flip, whether with `ReadDirection` enum it apply to both. > > >> >>>>> > > >> >>>>> The only issue I've found while reviewing the KIP is that > `all()` > > >> won't > > >> >>>> fit > > >> >>>>> within this approach. > > >> >>>>> > > >> >>>>> We could remove it from the KIP and argue that for WindowStore, > > >> >>>>> `fetchAll(0, Long.MAX_VALUE)` can be used to get all in reverse > > >> order, > > >> >>>> and > > >> >>>>> for KeyValueStore, no ordering guarantees are provided. > > >> >>>>> > > >> >>>>> If there is consensus with this changes, I will go and update > the > > >> KIP. > > >> >>>>> > > >> >>>>> On Thu, May 21, 2020 at 3:33 PM John Roesler < > vvcep...@apache.org> > > >> >>>> wrote: > > >> >>>>> > > >> >>>>>> Hi Jorge, > > >> >>>>>> > > >> >>>>>> Thanks for that idea. I agree, a feature flag would protect > anyone > > >> >>>>>> who may be depending on the current behavior. > > >> >>>>>> > > >> >>>>>> It seems better to locate the feature flag in the > initialization > > >> >>>> logic of > > >> >>>>>> the store, rather than have a method on the "live" store that > > >> changes > > >> >>>>>> its behavior on the fly. > > >> >>>>>> > > >> >>>>>> It seems like there are two options here, one is to add a new > > >> config: > > >> >>>>>> > > >> >>>>>> StreamsConfig.ENABLE_BACKWARDS_ITERATION = > > >> >>>>>> "enable.backwards.iteration > > >> >>>>>> > > >> >>>>>> Or we can add a feature flag in Materialized, like > > >> >>>>>> > > >> >>>>>> Materialized.enableBackwardsIteration() > > >> >>>>>> > > >> >>>>>> I think I'd personally lean toward the config, for the > following > > >> >>>> reason. > > >> >>>>>> The concern that Sophie raised is that someone's program may > depend > > >> >>>>>> on the existing contract of getting an empty iterator. We don't > > >> want > > >> >>>> to > > >> >>>>>> switch behavior when they aren't expecting it, so we provide > them a > > >> >>>>>> config to assert that they _are_ expecting the new behavior, > which > > >> >>>>>> means they take responsibility for updating their code to > expect > > >> the > > >> >>>> new > > >> >>>>>> behavior. > > >> >>>>>> > > >> >>>>>> There doesn't seem to be a reason to offer a choice of > behaviors > > >> on a > > >> >>>>>> per-query, or per-store basis. We just want people to be not > > >> surprised > > >> >>>>>> by this change in general. > > >> >>>>>> > > >> >>>>>> What do you think? > > >> >>>>>> Thanks, > > >> >>>>>> -John > > >> >>>>>> > > >> >>>>>> On Wed, May 20, 2020, at 17:37, Jorge Quilcate wrote: > > >> >>>>>>> Thank you both for the great feedback. > > >> >>>>>>> > > >> >>>>>>> I like the "fancy" proposal :), and how it removes the need > for > > >> >>>>>>> additional API methods. And with a feature flag on > `StateStore`, > > >> >>>>>>> disabled by default, should no break current users. > > >> >>>>>>> > > >> >>>>>>> The only side-effect I can think of is that: by moving the > flag > > >> >>>>> upwards, > > >> >>>>>>> all later operations become affected; which might be ok for > most > > >> >>>> (all?) > > >> >>>>>>> cases. I can't think of an scenario where this would be an > issue, > > >> >>>> just > > >> >>>>>>> want to point this out. > > >> >>>>>>> > > >> >>>>>>> If moving to this approach, I'd like to check if I got this > right > > >> >>>>> before > > >> >>>>>>> updating the KIP: > > >> >>>>>>> > > >> >>>>>>> - only `StateStore` will change by having a new method: > > >> >>>>>>> `backwardIteration()`, `false` by default to keep things > > >> compatible. > > >> >>>>>>> - then all `*Stores` will have to update their implementation > > >> based > > >> >>>> on > > >> >>>>>>> this flag. > > >> >>>>>>> > > >> >>>>>>> > > >> >>>>>>> On 20/05/2020 21:02, Sophie Blee-Goldman wrote: > > >> >>>>>>>>> There's no possibility that someone could be relying > > >> >>>>>>>>> on iterating over that range in increasing order, because > that's > > >> >>>> not > > >> >>>>>> what > > >> >>>>>>>>> happens. However, they could indeed be relying on getting an > > >> >>>> empty > > >> >>>>>>>> iterator > > >> >>>>>>>> > > >> >>>>>>>> I just meant that they might be relying on the assumption > that > > >> the > > >> >>>>>> range > > >> >>>>>>>> query > > >> >>>>>>>> will never return results with decreasing keys. The empty > > >> iterator > > >> >>>>>> wouldn't > > >> >>>>>>>> break that contract, but of course a surprise reverse > iterator > > >> >>>> would. > > >> >>>>>>>> > > >> >>>>>>>> FWIW I actually am in favor of automatically converting to a > > >> >>>> reverse > > >> >>>>>>>> iterator, > > >> >>>>>>>> I just thought we should consider whether this should be off > by > > >> >>>>>> default or > > >> >>>>>>>> even possible to disable at all. > > >> >>>>>>>> > > >> >>>>>>>> On Tue, May 19, 2020 at 7:42 PM John Roesler < > > >> vvcep...@apache.org > > >> >>>>> > > >> >>>>>> wrote: > > >> >>>>>>>> > > >> >>>>>>>>> Thanks for the response, Sophie, > > >> >>>>>>>>> > > >> >>>>>>>>> I wholeheartedly agree we should take as much into account > as > > >> >>>>> possible > > >> >>>>>>>>> up front, rather than regretting our decisions later. I > actually > > >> >>>> do > > >> >>>>>> share > > >> >>>>>>>>> your vague sense of worry, which was what led me to say > > >> initially > > >> >>>>>> that I > > >> >>>>>>>>> thought my counterproposal might be "too fancy". Sometimes, > it's > > >> >>>>>> better > > >> >>>>>>>>> to be explicit instead of "elegant", if we think more people > > >> >>>> will be > > >> >>>>>>>>> confused > > >> >>>>>>>>> than not. > > >> >>>>>>>>> > > >> >>>>>>>>> I really don't think that there's any danger of "relying on > a > > >> >>>> bug" > > >> >>>>>> here, > > >> >>>>>>>>> although > > >> >>>>>>>>> people certainly could be relying on current behavior. One > thing > > >> >>>> to > > >> >>>>> be > > >> >>>>>>>>> clear > > >> >>>>>>>>> about (which I just left a more detailed comment in > KAFKA-8159 > > >> >>>>> about) > > >> >>>>>> is > > >> >>>>>>>>> that > > >> >>>>>>>>> when we say something like key1 > key2, this ordering is > defined > > >> >>>> by > > >> >>>>>> the > > >> >>>>>>>>> serde's output and nothing else. > > >> >>>>>>>>> > > >> >>>>>>>>> Currently, thanks to your fix in > > >> >>>>>> https://github.com/apache/kafka/pull/6521 > > >> >>>>>>>>> , > > >> >>>>>>>>> the store contract is that for range scans, if from > to, > then > > >> >>>> the > > >> >>>>>> store > > >> >>>>>>>>> must > > >> >>>>>>>>> return an empty iterator. There's no possibility that > someone > > >> >>>> could > > >> >>>>> be > > >> >>>>>>>>> relying > > >> >>>>>>>>> on iterating over that range in increasing order, because > that's > > >> >>>> not > > >> >>>>>> what > > >> >>>>>>>>> happens. However, they could indeed be relying on getting an > > >> >>>> empty > > >> >>>>>>>>> iterator. > > >> >>>>>>>>> > > >> >>>>>>>>> My counterproposal was to actually change this contract to > say > > >> >>>> that > > >> >>>>>> the > > >> >>>>>>>>> store > > >> >>>>>>>>> must return an iterator over the keys in that range, but in > the > > >> >>>>>> reverse > > >> >>>>>>>>> order. > > >> >>>>>>>>> So, in addition to considering whether this idea is "too > fancy" > > >> >>>> (aka > > >> >>>>>>>>> confusing), > > >> >>>>>>>>> we should also consider the likelihood of breaking an > existing > > >> >>>>>> program with > > >> >>>>>>>>> this behavior/contract change. > > >> >>>>>>>>> > > >> >>>>>>>>> To echo your clarification, I'm also not advocating > strongly in > > >> >>>>> favor > > >> >>>>>> of my > > >> >>>>>>>>> proposal. I just wanted to present it for consideration > > >> alongside > > >> >>>>>> Jorge's > > >> >>>>>>>>> original one. > > >> >>>>>>>>> > > >> >>>>>>>>> Thanks for raising these very good points, > > >> >>>>>>>>> -John > > >> >>>>>>>>> > > >> >>>>>>>>> On Tue, May 19, 2020, at 20:49, Sophie Blee-Goldman wrote: > > >> >>>>>>>>>>> Rather than working around it, I think we should just fix > it > > >> >>>>>>>>>> Now *that's* a "fancy" idea :P > > >> >>>>>>>>>> > > >> >>>>>>>>>> That was my primary concern, although I do have a vague > sense > > >> of > > >> >>>>>> worry > > >> >>>>>>>>>> that we might be allowing users to get into trouble without > > >> >>>>>> realizing it. > > >> >>>>>>>>>> For example if their custom serdes suffer a similar bug as > the > > >> >>>>> above, > > >> >>>>>>>>>> and/or > > >> >>>>>>>>>> they rely on getting results in increasing order (of the > keys) > > >> >>>> even > > >> >>>>>> when > > >> >>>>>>>>>> to < from. Maybe they're relying on the fact that the range > > >> >>>> query > > >> >>>>>> returns > > >> >>>>>>>>>> nothing in that case. > > >> >>>>>>>>>> > > >> >>>>>>>>>> Not sure if that qualifies as relying on a bug or not, but > in > > >> >>>> that > > >> >>>>>> past > > >> >>>>>>>>>> we've > > >> >>>>>>>>>> taken the stance that we should not break compatibility > even if > > >> >>>> the > > >> >>>>>> user > > >> >>>>>>>>>> was relying on bugs or unintentional behavior. > > >> >>>>>>>>>> > > >> >>>>>>>>>> Just to clarify I'm not advocating strongly against this > > >> >>>> proposal, > > >> >>>>>> just > > >> >>>>>>>>>> laying > > >> >>>>>>>>>> out some considerations we should take into account. At > the end > > >> >>>> of > > >> >>>>>> the > > >> >>>>>>>>> day > > >> >>>>>>>>>> we should do what's right rather than maintain > compatibility > > >> >>>> with > > >> >>>>>>>>> existing > > >> >>>>>>>>>> bugs, but sometimes there's a reasonable middle ground. > > >> >>>>>>>>>> > > >> >>>>>>>>>> On Tue, May 19, 2020 at 6:15 PM John Roesler < > > >> >>>> vvcep...@apache.org> > > >> >>>>>>>>> wrote: > > >> >>>>>>>>>>> Thanks Sophie, > > >> >>>>>>>>>>> > > >> >>>>>>>>>>> Woah, that’s a nasty bug. Rather than working around it, I > > >> >>>> think > > >> >>>>> we > > >> >>>>>>>>> should > > >> >>>>>>>>>>> just fix it. I’ll leave some comments on the Jira. > > >> >>>>>>>>>>> > > >> >>>>>>>>>>> It doesn’t seem like it should be this KIP’s concern that > some > > >> >>>>>> serdes > > >> >>>>>>>>>>> might be incorrectly written. > > >> >>>>>>>>>>> > > >> >>>>>>>>>>> Were there other practical concerns that you had in mind? > > >> >>>>>>>>>>> > > >> >>>>>>>>>>> Thanks, > > >> >>>>>>>>>>> John > > >> >>>>>>>>>>> > > >> >>>>>>>>>>> On Tue, May 19, 2020, at 19:10, Sophie Blee-Goldman wrote: > > >> >>>>>>>>>>>> I like this "fancy idea" to just flip the to/from bytes > but I > > >> >>>>> think > > >> >>>>>>>>> there > > >> >>>>>>>>>>>> are some practical limitations to implementing this. In > > >> >>>>> particular > > >> >>>>>>>>>>>> I'm thinking about this issue > > >> >>>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-8159> with > the > > >> >>>>>> built-in > > >> >>>>>>>>>>> signed > > >> >>>>>>>>>>>> number serdes. > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> This trick would actually fix the problem for > > >> >>>> negative-negative > > >> >>>>>>>>> queries > > >> >>>>>>>>>>>> (ie where to & from are negative) but would cause > > >> undetectable > > >> >>>>>>>>>>>> incorrect results for negative-positive queries. For > example, > > >> >>>> say > > >> >>>>>> you > > >> >>>>>>>>>>>> call #range with from = -1 and to = 1, using the Short > > >> serdes. > > >> >>>>> The > > >> >>>>>>>>>>>> serialized bytes for that are > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> from = 1111111111111111 > > >> >>>>>>>>>>>> to = 0000000000000001 > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> so we would end up flipping those and iterating over all > keys > > >> >>>>> from > > >> >>>>>>>>>>>> 0000000000000001 to 1111111111111111. Iterating in > > >> >>>>> lexicographical > > >> >>>>>>>>>>>> order means we would iterate over every key in the space > > >> >>>> *except* > > >> >>>>>> for > > >> >>>>>>>>>>>> 0, but 0 is actually the *only* other key we meant to be > > >> >>>> included > > >> >>>>>> in > > >> >>>>>>>>> the > > >> >>>>>>>>>>>> range query. > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> Currently we just log a warning and return an empty > iterator > > >> >>>> when > > >> >>>>>>>>>>>> to < from, which is obviously also incorrect but feels > > >> >>>> slightly > > >> >>>>>> more > > >> >>>>>>>>>>>> palatable. If we start automatically converting to > reverse > > >> >>>>> queries > > >> >>>>>> we > > >> >>>>>>>>>>>> can't even log a warning in this case unless we wanted > to log > > >> >>>> a > > >> >>>>>>>>> warning > > >> >>>>>>>>>>>> every time, which would be weird to do for a valid usage > of a > > >> >>>> new > > >> >>>>>>>>>>>> feature. > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> All that said, I still like the idea overall. Off the > top of > > >> >>>> my > > >> >>>>>> head > > >> >>>>>>>>> I > > >> >>>>>>>>>>> guess > > >> >>>>>>>>>>>> we could add a store config to enable/disable automatic > > >> >>>> reverse > > >> >>>>>>>>>>> iteration, > > >> >>>>>>>>>>>> which is off by default? > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> Thanks for the KIP! This will be a nice addition > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> Sophie > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> > > >> >>>>>>>>>>>> On Tue, May 19, 2020 at 3:21 PM John Roesler < > > >> >>>>> vvcep...@apache.org> > > >> >>>>>>>>>>> wrote: > > >> >>>>>>>>>>>>> Hi there Jorge, > > >> >>>>>>>>>>>>> > > >> >>>>>>>>>>>>> Thanks for the KIP! > > >> >>>>>>>>>>>>> > > >> >>>>>>>>>>>>> I think this feature sounds very reasonable. > > >> >>>>>>>>>>>>> > > >> >>>>>>>>>>>>> I'm not 100% sure if this is "too fancy", but what do > you > > >> >>>> think > > >> >>>>>>>>>>>>> about avoiding the enum by instead allowing people to > flip > > >> >>>>>>>>>>>>> the "from" and "to" endpoints? I.e., reading from "A" > to "Z" > > >> >>>>> would > > >> >>>>>>>>>>>>> be a forward scan, and from "Z" to "A" would be a > backward > > >> >>>> one? > > >> >>>>>>>>>>>>> > > >> >>>>>>>>>>>>> Thanks, > > >> >>>>>>>>>>>>> -John > > >> >>>>>>>>>>>>> > > >> >>>>>>>>>>>>> On Tue, May 19, 2020, at 16:20, Jorge Quilcate wrote: > > >> >>>>>>>>>>>>>> Hi everyone, > > >> >>>>>>>>>>>>>> > > >> >>>>>>>>>>>>>> I would like to start the discussion for KIP-617: > > >> >>>>>>>>>>>>>> > > >> >>>>>>>>> > > >> >>>>>> > > >> >>>>> > > >> >>>> > > >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-617%3A+Allow+Kafka+Streams+State+Stores+to+be+iterated+backwards > > >> >>>>>>>>>>>>>> Looking forward to your feedback. > > >> >>>>>>>>>>>>>> > > >> >>>>>>>>>>>>>> Thanks! > > >> >>>>>>>>>>>>>> Jorge. > > >> >>>>>>>>>>>>>> > > >> >>>>>>>>>>>>>> > > >> >>>>>>>>>>>>>> Attachments: > > >> >>>>>>>>>>>>>> * 0x5F2C6E22064982DF.asc > > >> >>>>>>> > > >> >>>>>>> > > >> >>>>>>> Attachments: > > >> >>>>>>> * 0x5F2C6E22064982DF.asc > > >> >>>>>> > > >> >>>>> > > >> >>>> > > >> >>> > > >> >> > > >> > > >> > > >