Hi Patrick,

Sorry, I need to clarify.
In this current version of KIP in wiki, topic object are created with
constructor where driver, topicName and serdes are provided.

TestInputTopic<String, String> inputTopic = new TestInputTopic<String,
String>(testDriver, INPUT_TOPIC, new Serdes.StringSerde(), new
Serdes.StringSerde());

So if TopologyTestDriver modified, this could be

TestInputTopic<String, String> inputTopic =
testDriver.getInputTopic(INPUT_TOPIC, new Serdes.StringSerde(), new
Serdes.StringSerde());

or preferrable if serders can be found:

TestInputTopic<String, String> inputTopic =
testDriver.getInputTopic(INPUT_TOPIC);

This initialization done normally in test setup and after it can be used
with topic object:

inputTopic.pipeInput("Hello");


Or did you mean something else?

Jukka




to 9. toukok. 2019 klo 15.14 Patrik Kleindl (pklei...@gmail.com) kirjoitti:

> Hi Jukka
> Regarding your comment
> > If there would be a way to find out needed serders for the topic, it
> would make API even simpler.
> I was wondering if it wouldn't make more sense to have a "topic object"
> including the Serdes and use this instead of only passing in the name as a
> string everywhere.
> From a low-level perspective Kafka does and should not care what is inside
> the topic, but from a user perspective this information usually belongs
> together.
> Sidenote: Having topics as objects would probably also make it easier to
> track them from the outside.
> regards
> Patrik
>
> On Thu, 9 May 2019 at 10:39, Jukka Karvanen <jukka.karva...@jukinimi.com>
> wrote:
>
> > Hi,
> >
> > I will write new KIP for the TestTopologyDriver Input and Output
> usability
> > changes.
> > It is out of the scope of the current title: "Helper classes to make it
> > simpler to write test logic with TopologyTestDriver"
> > and we can get back to this KIP if that alternative is not favored.
> >
> > So my original approach was not to modify existing classes, but if we end
> > up modifing TTD, I would also change the
> > way to instantiate these topics. We could add getInputTopic("my-topic") /
> > getOutputTopic("my-topic") to TTD, so it would work
> > same way as with getStateStore and related methods.
> >
> > If there would be a way to find out needed serders for the topic, it
> would
> > make API even simpler.
> >
> > Generally still as a end user, I would prefer not only swapping the
> > ConsumerRecord and ProducerRecord, but having
> > interface accepting and returning Record, not needing to think about are
> > those ConsumerRecord or ProducerRecords.
> > and that way would could use same classes to pipe in and assert the
> > result.Something similar than  "private final static class Record"
> > in TopologyTestDriverTest.
> >
> > Jukka
> >
> > ke 8. toukok. 2019 klo 17.01 John Roesler (j...@confluent.io) kirjoitti:
> >
> > > Hi Jukka, thanks for the reply!
> > >
> > > I think this is a good summary (the discussion was getting a little
> > > unwieldy. I'll reply inline.
> > >
> > > Also, thanks for clarify about your library vs. this KIP. That makes
> > > perfect sense to me.
> > > >
> > > > 1. Add JavaDoc for KIP
> > > >
> > > > Is there a good example of KIP where Javadoc is included, so I can
> > > follow?
> > > > I create this KIP based on this as an example::
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-247%3A+Add+public+test+utils+for+Kafka+Streams
> > > >
> > > >
> > > > Now added some comments to KIP page to clarify timestamp handling,
> but
> > I
> > > > did not want to add full JavaDoc of each methods.
> > > > Is that enough?
> > >
> > > That's fine. I was just trying to make the review process more
> > > efficient for other reviewers (which makes getting your KIP accepted
> > > more efficient). I reviewed a few recent KIPs, and, indeed, I see that
> > > javadocs are not actually as common as I thought.
> > >
> > > > 2. TTD usability changes and swapping ConsumerRecord and
> ProducerRecord
> > > in
> > > > APIs
> > > >
> > > > To my point of view only:
> > > > - changing readRecord to return ConsumerRecord would cause we cannot
> > use
> > > > OutputVerifier
> > >
> > > Yes, we'd likely have to provide new methods in OutputVerifier to work
> > > with ConsumerRecord. If you buy into the plan of deprecating most of
> > > the current-style interactions, this wouldn't be that confusing, since
> > > all the ProducerRecord verifications would be deprecated, and only the
> > > ConsumerRecord verifications would remain "live".
> > >
> > > > - changing pipeInput to take in ProducerRecord, but not providing
> easy
> > > way
> > > > to contruct those like ConsumerRecordFactory
> > >
> > > I didn't follow this as well. The ConsumerRecordFactory is there
> > > because it's a pain to construct ConsumerRecords. Conversely,
> > > ProducerRecord has many convenience constructors, so we wouldn't need
> > > a factory at all. This is a net win for users, since there's less
> > > surface area for them to deal with. Under my proposal, we'd deprecate
> > > the whole ConsumerRecordFactory.
> > >
> > > Note that there's an "idea parity check" here: ConsumerRecords are
> > > hard to construct because developers aren't meant to ever construct
> > > them. They are meant to construct ProducerRecords, which is why it's
> > > made easy. TTD has inverted the relationships of these classes, which
> > > is why the ConsumerRecordFactory is necessary, but if we correct it,
> > > and return to a "normal" interaction with the Client library, then we
> > > don't need special support classes.
> > >
> > > > - if initializing ConsumerRecord to/from  ProducerRecord  in these
> > > classes
> > > > field by field contructor, there are risk new fields are not added to
> > > this
> > > > classes if there are changes in ProducerRecord or ConsumerRecord
> > >
> > > This risk seems pretty low, to be honest. We will have tests that
> > > exercise this testing framework, so if anyone changes ProducerRecord
> > > or ConsumerRecord, our tests will break. Since both libraries are
> > > build together, the problem would be fixed before the change is ever
> > > merged to trunk.
> > >
> > > > I would propose a separate KIP for these and probably other
> > > enhanchements:
> > > > -superclass or common interface for ConsumerRecord and ProducerRecord
> > > > -contructors to ConsumerRecord and ProducerRecord to initialize with
> > this
> > > > superclass
> > > > -modify OutputVerifier to work with both ConsumerRecord and
> > > ProducerRecord
> > > > -create new RecordFactory to replace ConsumerRecordFactory
> > >
> > > I understand your motivation to control the scope of this change, but
> > > I actually think that it's better for user-facing design changes to
> > > occur in fewer, bigger, chunks, rather than many small changes. People
> > > will get fatigued if multiple releases in a row change the
> > > test-support library from under their feet. Better to do it in one
> > > shot.
> > >
> > > Plus, this is a design discussion. We need to include the whole scope
> > > of the system in the design, or we may realize in Phase 3 that there
> > > was some design error in Phase 1, since we were only designing
> > > locally. This doesn't mean that we only need one Jira ticket, there
> > > can be many in support of this KIP, or that we only need one PR, it's
> > > certainly better to send multiple small PRs to decrease risk and ease
> > > reviews. But the design discussion doesn't need to be fragmented
> > > similarly.
> > >
> > > > 3. return null vs NoSuchElementException when empty queue
> > > >
> > > > Should this be also included to the above TTD usability changes?
> > > > If single row read methods is changed to throw expectiong, it would
> > > require
> > > > addition of hasRecords to able to verified the empty queue scenarios.
> > > > I do not know how to implement it currently without modifying TTD to
> > > > provide some kind way to get the queue size or peak items.
> > >
> > > Yes, it's absolutely within bounds to propose changes to TTD to
> > > support the ergonomic API you're proposing.
> > >
> > > > 4. IllegalArgumentException("topic doesn't exist")
> > > > Is this worth separate ticket?
> > >
> > > This is your call. That was just an idea in response to your
> experience.
> > >
> > > > 5. org.apache.kafka.streams.test vs org.apache.kafka.streams
> > > >
> > > > I was thinking org.apache.kafka.streams.test where also
> OutputVerifier
> > > and
> > > > ConsumerRecordFactory exist would be more logical place, but
> > > > I do not know is there some technical reasons why TTD are in
> > > > org.apache.kafka.streams, not in org.apache.kafka.streams.test where
> > > other
> > > > classes are.
> > > >
> > > > Did I skip something?
> > >
> > > Ah, no, you're right. I'm not sure why that is. I admit it's
> > > confusing. I don't think the package matters *that* much, just keep it
> > > wherever you think is appropriate.
> > >
> > >
> > > That's all! Thanks for entertaining my thoughts.
> > > -John
> > >
> >
>

Reply via email to