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