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 <[email protected]> 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 <[email protected]> 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 <[email protected]> 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 <[email protected]> > >>> 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 < > >>> [email protected]> > >>>>>> 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 <[email protected]> > >>> wrote: > >>>>>>> > >>>>>>>> Bumping this up again, thanks! > >>>>>>>> > >>>>>>>> On Fri, Apr 5, 2019, 14:36 Yishun Guan <[email protected]> wrote: > >>>>>>>> > >>>>>>>>> Hi, bumping this up again. Thanks! > >>>>>>>>> > >>>>>>>>> On Tue, Apr 2, 2019, 13:07 Yishun Guan <[email protected]> > >> 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 > >>>>>>>>>> > >>>>>>>>> > >>>>>>>> > >>>>>>> > >>>>>> > >>>>>> > >>>>> > >>>> > >>> > >> > > >
