Hi Matthias, That makes sense to me! I updated the code, definitely want to get your perspective on whether or not we want to support the `createFixedKeyRecord(Record)` method, since we already have the `TestRecord` defined in the utils, it might be cleaner to just support the `createFixedKeyRecord(TestRecord)` method.
For reference - https://github.com/s7pandey/kafka/commit/8ac92509d455d8175381a9b4c83900218941bf05#diff-2a3e6e23894a888e8c2fa486e2330f42b8fb28fe2216ba182e27d3d14958457b Also, looks like I do not have access to update the KIP, my confluence account is active now (s7pandey) but I think I need some permissions on the actual KIP page. Shashwat On Wed, Jun 12, 2024 at 8:32 PM Matthias J. Sax <mj...@apache.org> wrote: > I believe the class name was picked on purpose, to make clear that it > should not be used -- the problem is, that the class is in a public > package and is by itself public (that's unfortunately require, given how > Java works). > > Of course, it's also in the JavaDocs that the class is internal and > should not be used, but not everyone reads the JavaDocs necessarily, so > making it part of the name makes it much more explicit, what I believe > is a good thing. > > I would consider it a fix/improvement, if we could exclude > `InternalFixedKeyRecordFactory` from JavaDoc generation during the > release build -- but I don't think we need a KIP for this, as I would > rather consider it a bug-fix to exclude an internal class in the > JavaDocs build step. > > > -Matthias > > On 6/12/24 4:47 PM, Shashwat Pandey wrote: > > Hi Matthias, > > > > I think that strategy definitely works, abstracting away changes to > > FixedKeyRecord from users, I can put that new factory class and update > the > > KIP. > > > > This might be a discussion for another KIP, but would it also make sense > to > > rename the > > InternalFixedKeyRecordFactory to just FixedKeyRecordFactory also make > sense? > > > > > > Regards, > > Shashwat Pandey > > > > > > On Mon, Jun 10, 2024 at 5:07 PM Matthias J. Sax <mj...@apache.org> > wrote: > > > >> Shaswhat, > >> > >> any updates on this KIP? -- I still think that recommending to use > >> `InternalFixedKeyRecordFactory` is not the best way to write test code. > >> Changing `FixedKeyRecord` constructor (as I mentioned in my last email) > >> might not be a good solution either. > >> > >> Maybe a cleaner way would be (so sidestep this problem), to add a new > >> public "factory class" into the test package to generate > >> FixedKeyRecords, and this factory could internally use > >> `InternalFixedKeyRecordFactory`? It looks cleaner to me from an API POV, > >> and if we change anything how `FixedKeyRecord` can be created, it would > >> become a non-user-facing / internal change to the "factory" we provide. > >> > >> > >> -Matthias > >> > >> On 5/22/24 12:02 AM, Matthias J. Sax wrote: > >>> I was not aware of `InternalFixedKeyRecordFactory`. As the name > >>> indicates, it's considered an internal class, so not sure if we should > >>> recommend to use it in test... > >>> > >>> I understand why this class is required, and why it was put into a > >>> public package; the way Java works, enforces this. Not sure if we could > >>> find a better solution. > >>> > >>> Might be good to hear from others. > >>> > >>> > >>> -Matthias > >>> > >>> On 5/21/24 3:57 PM, Shashwat Pandey wrote: > >>>> Looking at the ticket and the sample code, I think it would be > >>>> possible to > >>>> continue using `InternalFixedKeyRecordFactory` as the avenue to create > >>>> `FixedKeyRecord`s in tests. As long as there was a > >>>> MockFixedKeyProcessorContext, I think we would be able to test > >>>> FixedKeyProcessors without a Topology. > >>>> > >>>> I created a sample repo with the `MockFixedKeyProcessorContext` here > is > >>>> what I think the tests would look like: > >>>> > >> > https://github.com/s7pandey/kafka-processor-tests/blob/main/src/test/java/com/example/demo/MyFixedKeyProcessorTest.java > >>>> > >>>> > >>>> > >>>> On Mon, May 20, 2024 at 9:05 PM Matthias J. Sax <mj...@apache.org> > >> wrote: > >>>> > >>>>> Had a discussion on > https://issues.apache.org/jira/browse/KAFKA-15242 > >>>>> and it was pointed out, that we also need to do something about > >>>>> `FixedKeyRecord`. It does not have a public constructor (what is > >>>>> correct; it should not have one). However, this makes testing > >>>>> `FixedKeyProcessor` impossible w/o extending `FixedKeyRecord` > manually > >>>>> what does not seem to be right (too clumsy). > >>>>> > >>>>> It seems, we either need some helper builder method (but not clear to > >> me > >>>>> where to add it in an elegant way) which would provide us with a > >>>>> `FixedKeyRecord`, or add some sub-class to the test-utils module > which > >>>>> would extend `FixedKeyRecord`? -- Or maybe an even better solution? I > >>>>> could not think of something else so far. > >>>>> > >>>>> > >>>>> Thoughts? > >>>>> > >>>>> > >>>>> On 5/3/24 9:46 AM, Matthias J. Sax wrote: > >>>>>> Please also update the KIP. > >>>>>> > >>>>>> To get a wiki account created, please request it via a commet on > this > >>>>>> ticket: https://issues.apache.org/jira/browse/INFRA-25451 > >>>>>> > >>>>>> After you have the account, please share your wiki id, and we can > give > >>>>>> you write permission on the wiki. > >>>>>> > >>>>>> > >>>>>> > >>>>>> -Matthias > >>>>>> > >>>>>> On 5/3/24 6:30 AM, Shashwat Pandey wrote: > >>>>>>> Hi Matthias, > >>>>>>> > >>>>>>> Sorry this fell out of my radar for a bit. > >>>>>>> > >>>>>>> Revisiting the topic, I think you’re right and we accept the > >>>>>>> duplicated > >>>>>>> nesting as an appropriate solution to not affect the larger public > >>>>>>> API. > >>>>>>> > >>>>>>> I can update my PR with the change. > >>>>>>> > >>>>>>> Regards, > >>>>>>> Shashwat Pandey > >>>>>>> > >>>>>>> > >>>>>>> On Wed, May 1, 2024 at 11:00 PM Matthias J. Sax <mj...@apache.org> > >>>>> wrote: > >>>>>>> > >>>>>>>> Any updates on this KIP? > >>>>>>>> > >>>>>>>> On 3/28/24 4:11 AM, Matthias J. Sax wrote: > >>>>>>>>> It seems that `MockRecordMetadata` is a private class, and thus > not > >>>>>>>>> part > >>>>>>>>> of the public API. If there are any changes required, we don't > >>>>>>>>> need to > >>>>>>>>> discuss on the KIP. > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> For `CapturedPunctuator` and `CapturedForward` it's a little bit > >>>>>>>>> more > >>>>>>>>> tricky. My gut feeling is, that the classes might not need to be > >>>>>>>>> changed, but if we use them within `MockProcessorContext` and > >>>>>>>>> `MockFixedKeyProcessorContext` it might be weird to keep the > >> current > >>>>>>>>> nesting... The problem I see is, that it's not straightforward > >>>>>>>>> how to > >>>>>>>>> move the classes w/o breaking compatibility, nor if we duplicate > >>>>>>>>> them as > >>>>>>>>> standalone classes w/o a larger "splash radius". (We would need > >>>>>>>>> to add > >>>>>>>>> new overloads for MockProcessorContext#scheduledPunctuators() and > >>>>>>>>> MockProcessorContext#forwarded()). > >>>>>>>>> > >>>>>>>>> Might be good to hear from others if we think it's worth this > >> larger > >>>>>>>>> changes to get rid of the nesting, or just accept the somewhat > not > >>>>>>>>> ideal > >>>>>>>>> nesting as it technically is not a real issue? > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> -Matthias > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> On 3/15/24 1:47 AM, Shashwat Pandey wrote: > >>>>>>>>>> Thanks for the feedback Matthias! > >>>>>>>>>> > >>>>>>>>>> The reason I proposed the extension of MockProcessorContext was > >>>>>>>>>> more > >>>>>>>>>> to do > >>>>>>>>>> with the internals of the class (MockRecordMetadata, > >>>>>>>>>> CapturedPunctuator and > >>>>>>>>>> CapturedForward). > >>>>>>>>>> > >>>>>>>>>> However, I do see your point, I would then think to split > >>>>>>>>>> MockProcessorContext and MockFixedKeyProcessorContext, some of > the > >>>>>>>>>> internal > >>>>>>>>>> classes should also be extracted i.e. MockRecordMetadata, > >>>>>>>>>> CapturedPunctuator and probably a new CapturedFixedKeyForward. > >>>>>>>>>> > >>>>>>>>>> Let me know what you think! > >>>>>>>>>> > >>>>>>>>>> > >>>>>>>>>> Regards, > >>>>>>>>>> Shashwat Pandey > >>>>>>>>>> > >>>>>>>>>> > >>>>>>>>>> On Mon, Mar 11, 2024 at 10:09 PM Matthias J. Sax < > >> mj...@apache.org> > >>>>>>>>>> wrote: > >>>>>>>>>> > >>>>>>>>>>> Thanks for the KIP Shashwat. Closing this testing gap is great! > >> It > >>>>>>>>>>> did > >>>>>>>>>>> come up a few time already... > >>>>>>>>>>> > >>>>>>>>>>> One question: why do you propose to `extend > >> MockProcessorContext`? > >>>>>>>>>>> > >>>>>>>>>>> Given how the actual runtime context classes are setup, it > seems > >>>>> that > >>>>>>>>>>> the regular context and fixed-key-context are distinct, and > thus > >> I > >>>>>>>>>>> believe both mock-context classes should be distinct, too? > >>>>>>>>>>> > >>>>>>>>>>> What I mean is that FixedKeyProcessorContext does not extend > >>>>>>>>>>> ProcessorContext. Both classes have a common parent > >>>>> ProcessINGContext > >>>>>>>>>>> (note the very similar but different names), but they are > >>>>>>>>>>> "siblings" > >>>>>>>>>>> only, so why make the mock processor a parent-child > relationship? > >>>>>>>>>>> > >>>>>>>>>>> It seems better to do > >>>>>>>>>>> > >>>>>>>>>>> public class MockFixedKeyProcessorContext<KForward, VForward> > >>>>>>>>>>> implements FixedKeyProcessorContext<KForward, VForward>, > >>>>>>>>>>> RecordCollector.Supplier > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> Of course, if there is code we can share between both > >> mock-context > >>>>> we > >>>>>>>>>>> should so this, but it should not leak into the public API? > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> -Matthias > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> On 3/11/24 5:21 PM, Shashwat Pandey wrote: > >>>>>>>>>>>> Hi everyone, > >>>>>>>>>>>> > >>>>>>>>>>>> I would like to start the discussion on > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>> > >>>>> > >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1027%3A+Add+MockFixedKeyProcessorContext > >>>>>>>>>>>> > >>>>>>>>>>>> This adds MockFixedKeyProcessorContext to the Kafka Streams > Test > >>>>>>>>>>>> Utils > >>>>>>>>>>>> library. > >>>>>>>>>>>> > >>>>>>>>>>>> Regards, > >>>>>>>>>>>> Shashwat Pandey > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>> > >>>>>>> > >>>>> > >>>> > >>>> > >> > > > -- Regards, Shashwat Pandey 647-453-1102