What is the status of this KIP? Any updates?
-Matthias On 9/10/19 8:08 PM, Sophie Blee-Goldman wrote: > Just took a look at the current KIP, and I think you should actually be > fine if you're just mocking the stores. > The issue I brought up isn't necessarily blocking this KIP, but it is > related -- just wanted to bring it up and > see if there's any overlap, or if it's better to address separately. > > The problem isn't actually with in-memory stores (as opposed to > persistent), I suspect > people just happen to have exclusively hit/reported this issue with > in-memory stores since > a lightweight in-memory store is more attractive for unit tests than a > bunch of RocksDB instances > > The current problem technically only affects window/session stores, but the > workaround for KV stores > is not necessarily stable or supported. The issue is that to create a > store, you must use the store Supplier/Builder > provided in the public API (e.g. Stores#windowStoreBuilder), which requires > `init` to be called before using the store. > `init` takes a ProcessorContext as a parameter, and for KV stores you can > just pass in a MockProcessorContext. > Unfortunately, window/session stores internally cast the ProcessorContext > to an InternalProcessorContext in order > to set up some metrics, so you end up with a ClassCastException and no way > to use window/session stores in your test. > > So if you're literally wrapping any of these stores (eg > InMemoryWindowStore, or RocksDBSessionStore) then I think > you actually would run into this. > > Anyways, the current state of things is that we don't really support using > state stores in unit testing at all -- you can't > record the number of expected put/get calls, and you can't use an actual > store to, well, store things. We definitely > need both of these things to really round out our unit tests, but we don't > need to solve both of them in one KIP. > Probably best to avoid the issue in this KIP if possible :) > > On Thu, Sep 5, 2019 at 11:23 AM Yishun Guan <gyis...@gmail.com> wrote: > >> Thanks Sophie! >> >> I took a look at the issue and the mailing thread. So in other words, >> people are having issues writing unit tests using in-memory stores >> (which is a very common practice due to the lack of a better >> alternative), so we try to provide a better solution for testings, and >> hopefully works well with the current MockProcessContext. But the >> current issues we are facing with the in-memory stores, how can we >> better fix them in the mock stores? Should I think more about how the >> mock stores will interact with MockProcessorContext? The design I >> present now it's just a wrapper on a store. Do you think we need to >> address that before we go further? Instead of a wrapper, should we >> think about building a more comprehensive mock store? >> >> Thanks, >> Yishun >> >> On Thu, Aug 29, 2019 at 12:18 AM Sophie Blee-Goldman >> <sop...@confluent.io> wrote: >>> >>> Hey Yishun! Glad to see this is in the works :) >>> >>> Within the past month or so, needing state stores for unit tests has been >>> brought up multiple times. Unfortunately, before now some people had to >>> rely on internal APIs to get a store for their tests, which is unsafe as >>> they can (and in this case >>> < >> https://mail-archives.apache.org/mod_mbox/kafka-users/201907.mbox/%3cCAM0Vdef0h3p4gB=r3s=vvgssqqzqa4oxxkpl5cnpaxn146p...@mail.gmail.com%3e >>> , >>> did) change. While there is an unstable workaround for KV stores, there >> is >>> unfortunately no good way to get a window or session store for your >> tests. This >>> ticket <https://issues.apache.org/jira/browse/KAFKA-8630> explains that >>> particular issue, plus some ways to resolve it that could get kind of >> messy. >>> >>> I think that ticket would likely be subsumed by your KIP (and much >>> cleaner), but I just wanted to point to some use cases and make sure we >>> have them covered within this KIP. We definitely have a gap here and I >>> think it's pretty clear many users would benefit from state store support >>> in unit tests! >>> >>> Cheers, >>> Sophie >>> >>> On Tue, Aug 27, 2019 at 1:11 PM Yishun Guan <gyis...@gmail.com> wrote: >>> >>>> Hi All, >>>> >>>> I have finally worked on this KIP again and want to discuss with you >>>> all before this KIP goes dormant. >>>> >>>> Recap: https://issues.apache.org/jira/browse/KAFKA-6460 >>>> >>>> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-448%3A+Add+State+Stores+Unit+Test+Support+to+Kafka+Streams+Test+Utils >>>> >>>> I have updated my KIP. >>>> 1. Provided an example of how the test will look. >>>> 2. Allow the tester to use their StateStore of choice as a backend >>>> store when testing. >>>> 3. Argument against EasyMock: for now, I don't really have a strong >>>> point against EasyMock. If people are comfortable with EasyMock and >>>> think building a full tracking/capturing stateStore is heavyweight, >>>> this makes sense to me too, and we can put this KIP as `won't >>>> implement`. >>>> >>>> >>>> I also provided a proof of concept PR for review: >>>> https://github.com/apache/kafka/pull/7261/files >>>> >>>> Thanks, >>>> Yishun >>>> >>>> On Tue, Apr 30, 2019 at 4:03 AM Matthias J. Sax <matth...@confluent.io >>> >>>> wrote: >>>>> >>>>> I just re-read the discussion on the original Jira. >>>>> >>>>> It's still a little unclear to me, how this should work end-to-end? >> It >>>>> would be good, to describe some test patterns that we want to support >>>>> first. Maybe using some examples, that show how a test would be >> written? >>>>> >>>>> I don't think that we should build a whole mocking framework similar >> to >>>>> EasyMock (or others); why re-invent the wheel? I think the goal >> should >>>>> be, to allow people to use their mocking framework of choice, and to >>>>> easily integrate it with `TopologyTestDriver`, without the need to >>>>> rewrite the code under test. >>>>> >>>>> >>>>> For the currently internal `KeyValueStoreTestDriver`, it's seems to >> be a >>>>> little different, as the purpose of this driver is to test a store >>>>> implementation. Hence, most users won't need this, because they use >> the >>>>> built-in stores anyway, ie, this driver would be for advanced users >> that >>>>> build their own stores. >>>>> >>>>> I think it's actually two orthogonal things and it might even be >> good to >>>>> split both into two KIPs. >>>>> >>>>> >>>>> >>>>> -Matthias >>>>> >>>>> >>>>> On 4/30/19 7:52 AM, Yishun Guan wrote: >>>>>> Sounds good! Let me work on this more and add some more >> information to >>>> this >>>>>> KIP before we continue. >>>>>> >>>>>> On Tue, Apr 30, 2019, 00:45 Bruno Cadonna <br...@confluent.io> >> wrote: >>>>>> >>>>>>> Hi Yishun, >>>>>>> >>>>>>> Thank you for continuing with this KIP. IMO, this KIP is very >>>> important to >>>>>>> develop robust code. >>>>>>> >>>>>>> I think, a good approach is to do some research on mock >> development >>>> on the >>>>>>> internet and in the literatures and then try to prototype the >> mocks. >>>> These >>>>>>> activities should yield you a list of pros and cons that you can >> add >>>> to the >>>>>>> KIP. With this information it is simpler for everybody to discuss >>>> this KIP. >>>>>>> >>>>>>> Does this make sense to you? >>>>>>> >>>>>>> Best, >>>>>>> Bruno >>>>>>> >>>>>>> On Mon, Apr 29, 2019 at 7:11 PM Yishun Guan <gyis...@gmail.com> >>>> wrote: >>>>>>> >>>>>>>> Hi, >>>>>>>> >>>>>>>> Sorry for the late reply, I have read through all your valuable >>>>>>>> comments. The KIP still needs work at this point. >>>>>>>> >>>>>>>> I think at this point, one question comes up is that, how should >> we >>>>>>>> implement the mock stores - as Sophie suggested, should we open >> to >>>> all >>>>>>>> Store backend and just wrap around the Store class type which the >>>> user >>>>>>>> will be providing - or, as Bruno suggested, we shouldn't have a >>>>>>>> production backend store to be wrapped around in a mock store, >> just >>>>>>>> keep track of the state of each method calls, even EasyMock >> could be >>>>>>>> one of the option too. >>>>>>>> >>>>>>>> Personally, EasyMock will makes the implementation easier but >>>> building >>>>>>>> from scratch provides extra functionality and provides >> expandability >>>>>>>> (But I am not sure what kind of extra functionality we want in >> the >>>>>>>> future). >>>>>>>> >>>>>>>> What do you guys think? >>>>>>>> >>>>>>>> Best, >>>>>>>> Yishun >>>>>>>> >>>>>>>> On Fri, Apr 26, 2019 at 2:03 AM Matthias J. Sax < >>>> matth...@confluent.io> >>>>>>>> wrote: >>>>>>>>> >>>>>>>>> What is the status of this KIP? >>>>>>>>> >>>>>>>>> >>>>>>>>> Btw: there is also KIP-456. I was wondering if it might be >> required >>>> or >>>>>>>>> helpful to align the design of both with each other. Thoughts? >>>>>>>>> >>>>>>>>> >>>>>>>>> >>>>>>>>> -Matthias >>>>>>>>> >>>>>>>>> >>>>>>>>> On 4/11/19 12:17 AM, Matthias J. Sax wrote: >>>>>>>>>> Thanks for the KIP. Only one initial comment (Sophie mentioned >> this >>>>>>>>>> already but I want to emphasize on it). >>>>>>>>>> >>>>>>>>>> You state that >>>>>>>>>> >>>>>>>>>>> These will be internal classes, so no public API/interface. >>>>>>>>>> >>>>>>>>>> If this is the case, we don't need a KIP. However, the idea of >> the >>>>>>>>>> original Jira is to actually make those classes public, as >> part of >>>>>>> the >>>>>>>>>> `streams-test-utils` package. If it's not public, developers >> should >>>>>>> not >>>>>>>>>> use them, because they don't have any backward compatibility >>>>>>>> guarantees. >>>>>>>>>> >>>>>>>>>> Hence, I would suggest that the corresponding classes go into >> a new >>>>>>>>>> package `org.apache.kafka.streams.state`. >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> -Matthias >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> On 4/9/19 8:58 PM, Bruno Cadonna wrote: >>>>>>>>>>> Hi Yishun, >>>>>>>>>>> >>>>>>>>>>> Thank you for the KIP. >>>>>>>>>>> >>>>>>>>>>> I have a couple of comments: >>>>>>>>>>> >>>>>>>>>>> 1. Could you please add an example to the KIP that >> demonstrates >>>> how >>>>>>>> the >>>>>>>>>>> mocks should be used in a test? >>>>>>>>>>> >>>>>>>>>>> 2. I am wondering, whether the MockKeyValueStore needs to be >>>> backed >>>>>>>> by an >>>>>>>>>>> actual KeyValueStore (in your KIP InMemoryKeyValueStore). >> Would it >>>>>>> not >>>>>>>>>>> suffice to provide the mock with the entries that it has to >> check >>>> in >>>>>>>> case >>>>>>>>>>> of input operation like put() and with the entries it has to >>>> return >>>>>>>> in case >>>>>>>>>>> of an output operation like get()? In my opinion, a mock >> should >>>> have >>>>>>>> as >>>>>>>>>>> little and as simple code as possible. A unit test should >> depend >>>> as >>>>>>>> little >>>>>>>>>>> as possible from productive code that it does not explicitly >> test. >>>>>>>>>>> >>>>>>>>>>> 3. I would be interested in the arguments against using a >>>>>>>> well-established >>>>>>>>>>> and well-tested mock framework like EasyMock. If there are >> good >>>>>>>> arguments, >>>>>>>>>>> they should be listed under 'Rejected Alternatives'. >>>>>>>>>>> >>>>>>>>>>> 3. What is the purpose of the parameter 'time' in >>>> MockStoreFactory? >>>>>>>>>>> >>>>>>>>>>> Best, >>>>>>>>>>> Bruno >>>>>>>>>>> >>>>>>>>>>> On Tue, Apr 9, 2019 at 11:29 AM Sophie Blee-Goldman < >>>>>>>> sop...@confluent.io> >>>>>>>>>>> wrote: >>>>>>>>>>> >>>>>>>>>>>> Hi Yishun, thanks for the KIP! I have a few initial >>>>>>>> questions/comments: >>>>>>>>>>>> >>>>>>>>>>>> 1) It may be useful to capture the iterator results as well >> (eg >>>>>>> with >>>>>>>> a >>>>>>>>>>>> MockIterator that wraps the underlying iterator and records >> the >>>>>>> same >>>>>>>> way >>>>>>>>>>>> the MockStore wraps/records the underlying store) >>>>>>>>>>>> >>>>>>>>>>>> 2) a. Where is the "persistent" variable coming from or being >>>> used? >>>>>>>> It >>>>>>>>>>>> seems the MockKeyValueStore accepts it in the constructor, >> but >>>> only >>>>>>>> the >>>>>>>>>>>> name parameter is passed when constructing a new >>>> MockKeyValueStore >>>>>>> in >>>>>>>>>>>> build() ... also, if we extend InMemoryXXXStore shouldn't >> this >>>>>>>> always be >>>>>>>>>>>> false? >>>>>>>>>>>> b. Is the idea to wrap an in-memory store for each type >>>>>>>> (key-value, >>>>>>>>>>>> session, etc)? We don't (yet) offer an in-memory version of >> the >>>>>>>> session >>>>>>>>>>>> store although it is in the works, so this will be possible >> -- I >>>> am >>>>>>>> more >>>>>>>>>>>> wondering if it makes sense to decide this for the user or to >>>> allow >>>>>>>> them to >>>>>>>>>>>> choose between in-memory or rocksDB by setting "persistent" >>>>>>>>>>>> >>>>>>>>>>>> 3) I'm wondering if users might want to be able to plug in >> their >>>>>>> own >>>>>>>> custom >>>>>>>>>>>> stores as the underlying backend...should we support this as >>>> well? >>>>>>>> WDYT? >>>>>>>>>>>> >>>>>>>>>>>> 4) We probably want to make these stores available through >> the >>>>>>> public >>>>>>>>>>>> test-utils package (maybe not the stores themselves which >> should >>>> be >>>>>>>>>>>> internal, but should there be some kind of public API that >> gives >>>>>>>> access to >>>>>>>>>>>> them?) >>>>>>>>>>>> >>>>>>>>>>>> Cheers, >>>>>>>>>>>> Sophie >>>>>>>>>>>> >>>>>>>>>>>> On Tue, Apr 9, 2019 at 9:19 AM Yishun Guan < >> gyis...@gmail.com> >>>>>>>> wrote: >>>>>>>>>>>> >>>>>>>>>>>>> Bumping this up again, thanks! >>>>>>>>>>>>> >>>>>>>>>>>>> On Fri, Apr 5, 2019, 14:36 Yishun Guan <gyis...@gmail.com> >>>> wrote: >>>>>>>>>>>>> >>>>>>>>>>>>>> Hi, bumping this up again. Thanks! >>>>>>>>>>>>>> >>>>>>>>>>>>>> On Tue, Apr 2, 2019, 13:07 Yishun Guan <gyis...@gmail.com> >>>>>>> wrote: >>>>>>>>>>>>>> >>>>>>>>>>>>>>> Hi All, >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> I like to start a discussion on KIP-448 >>>>>>>>>>>>>>> (https://cwiki.apache.org/confluence/x/SAeZBg). It is >> about >>>>>>>> adding >>>>>>>>>>>>>>> Mock state stores and relevant components for testing >>>> purposes. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Here is the JIRA: >>>>>>>> https://issues.apache.org/jira/browse/KAFKA-6460 >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> This is a rough KIP draft, review and comment are >> appreciated. >>>>>>> It >>>>>>>>>>>>>>> seems to be tricky and some requirements and details are >> still >>>>>>>> needed >>>>>>>>>>>>>>> to be discussed. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Thanks, >>>>>>>>>>>>>>> Yishun >>>>>>>>>>>>>>> >>>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>> >>>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >> >
signature.asc
Description: OpenPGP digital signature