Ok, I will modify KIP Public Interface in a wiki based on the feedback. TestRecordFactory / ConsumerRecordFactory was used by TestInputTopic with the version I had with KIP456, but maybe I can merge That functionality to InputTopic or TestRecordFactory can kept non public maybe moving it to internals package.
I will make the proposal with a slim down interface. I don't want to go to so slim as you proposed with only TestRecord or List<TestRecord>, because you then still end up doing helper methods to construct List of TestRecord. The list of values is easier to write and clearer to read than if you need to contruct list of TestRecords. For example: final List<String> inputValues = Arrays.asList( "Apache Kafka Streams Example", "Using Kafka Streams Test Utils", "Reading and Writing Kafka Topic" ); inputTopic.pipeValueList(inputValues); Let's check after the next iteration is it still worth reducing the methods. Jukka pe 14. kesäk. 2019 klo 18.27 John Roesler (j...@confluent.io) kirjoitti: > Thanks, Jukka, > > Ok, I buy this reasoning. > > Just to echo what I think I read, you would just drop ClientRecord > from the proposal, and TestRecord would stand on its own, with the > same methods and properties you proposed, and the "input topic" would > accept TestRecord, and the "output topic" would produce TestRecord? > Further, the "input and output topic" classes would internally handle > the conversion to and from ConsumerRecord and ProducerRecord to pass > to and from the TopologyTestDriver? > > This seems good to me. > > Since the object coming out of the "output topic" is much more > ergonomic, I suspect we won't need the OutputVerifier at all. It was > mostly needed because of all the extra junk in ProducerRecord you need > to ignore. It seems better to just deprecate it. If in the future it > turns out there is some actual use case for a verifier, we can have a > very small KIP to add one. But reading your response, I suspect that > existing test verification libraries would be sufficient on their own. > > Similarly, it seems like we can shrink the total interface by removing > the TestRecordFactory from the proposal. If TestRecord already offers > all the constructors you'd want, then the only benefit of the factory > is to auto-increment the timestamps, but then again, the "input topic" > can already do that (e.g., it can do it if the record timestamp is not > set). > > Likewise, if the TestRecords are easy to create, then we don't need > all the redundant methods in "input topic" to pipe values, or > key/values, or key/value/timestamp, etc. We can do with just two > methods, one for a single TestRecord and one for a collection of them. > This reduces API ambiguity and also dramatically decreases the surface > area of the interface, which ultimately makes it much easier to use. > > It's best to start with the smallest interface that will do the job > and expand it upon request, rather than throwing in everything you can > think of up front. The extra stuff would be confusing to people facing > two practically identical paths to accomplish the same goal, and it's > very difficult to slim the interface down later, because we don't > really know which parts are more popular (i.e., no one submits > "feature requests" to _remove_ stuff they don't need, only to _add_ > stuff that they need. > > But overall, I really like the structure of this design. I'm super > excited about this KIP. > > Thanks, > -John > > On Fri, Jun 14, 2019 at 2:55 AM Jukka Karvanen > <jukka.karva...@jukinimi.com> wrote: > > > > Hi, > > > > I am not a fan of swapping only ProducerRecord and ConsumerRecord. > > As a test writer point of view I do not want to care about the difference > > of those and > > that way I like to have object type which can be used to pipe records in > > and compare outputs. > > That way avoid unnecessary conversions between ProducerRecord and > > ConsumerRecord. > > > > My initial assumption was that ProducerRecord and ConsumerRecord.could > > implement the same ClientRecord > > and that way test writer could have used either of those, but seems that > > return type of timestamp method long vs Long is not compatible. > > Now the main advantage of ClientRecord is no need to duplicate > > OutputVerifier when it is modified from ProducerRecord to ClientRecord. > > Generally is there need for OutputVerifier. Can we deprecate the existing > > and use standard assertion libraries for new test. > > > > If you use hamcrest, assert-j or any other assertion library for the > rest > > of the test, why not use it with these also. > > When we have these methods to access only needed fields it is easier to > > write test like this: > > assertThat(outputTopic.readValue()).isEqualTo("Hello"); > > > > or > > assertThat(outputTopic.readRecord()).isEqualTo(expectedRecord); > > > > Only value for new OutputVerifier is when needing to ignore some fields > > ClientRecord actual = outputTopic.readRecord(); > > assertThat(actual.value()).isEqualTo("Hello"); > > assertThat(actual.key()).isEqualTo(expectedKey); > > assertThat(actual.timestamp()).isEqualTo(expectedTimestamp); > > > > So if want to leave client package untouched, I would modify the methods > > with ClientRecord now in InputTopic and OutputTopic to pass in and out > this > > TestRecord instead. > > In that case there would be possibility to add methods to TestRecord to > > help ignore some fields in assertions like: > > > > > assertThat(outputTopic.readRecord().getValueTimestamp()).isEqualTo(expectedRecord.get > > ValueTimestamp()); > > > > How about this alternative? > > If this way sounds better I will modify KIP page in wiki. > > > > > > Jukka > > > > > > to 13. kesäk. 2019 klo 18.30 John Roesler (j...@confluent.io) kirjoitti: > > > > > Hey, all, maybe we can jump-start this discussion. > > > > > > I think this approach would be very ergonomic for testing, and would > > > help reduce boilerplate in tests. > > > > > > The think I like most about it is that it mirrors the mental model > > > that people already have from Kafka Streams, in which you write to an > > > "input topic" and then get your results from an "output topic". As a > > > side benefit, the input and output topics in the proposal also close > > > over the serdes, which makes it much less boilerplate for test code. > > > > > > If I can offer one suggestion for change, I'm not sure I'm totally > > > sold on the need for a new abstraction "ClientRecord" with an > > > implementation for tests "TestRecord". It seems like this > > > unnecessarily complicates the main (non-testing) data model. It seems > > > like it would be sufficient, and just as ergonomic, to have the input > > > topic accept ProducerRecords and the output topic return > > > ConsumerRecords. I'm open to discussion on this point, though. > > > > > > Thanks for the proposal, Jukka! > > > -John > > > > > > On Mon, May 20, 2019 at 7:59 AM Jukka Karvanen > > > <jukka.karva...@jukinimi.com> wrote: > > > > > > > > Hi All, > > > > > > > > I would like to start the discussion on KIP-470: TopologyTestDriver > test > > > > input and output usability improvements: > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements > > > > > > > > > > > > This KIP is inspired by the Discussion in KIP-456: Helper classes to > make > > > > it simpler to write test logic with TopologyTestDriver: > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-456 > > > > > > > > %3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver > > > > > > > > > > > > The proposal in KIP-456 > > > > < > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver > > > > > > > > was > > > > to add alternate way to input and output topic, but this KIP enhance > > > those > > > > classes and deprecate old functionality to make clear interface for > test > > > > writer to use. > > > > > > > > In current KIP-470 proposal, topic objects are created with > topicName and > > > > related serders. > > > > public final <K, V> TestInputTopic<K, V> createInputTopic(final > > > String > > > > topicName, final Serde<K> keySerde, final Serde<V> valueSerde); > > > > public final <K, V> TestOutputTopic<K, V> createOutputTopic(final > > > String > > > > topicName, final Serde<K> keySerde, final Serde<V> valueSerde); > > > > One thing I wondered if there way to find out topic related serde > from > > > > TopologyTestDriver topology, it would simply creation of these Topic > > > > objects: > > > > public final <K, V> TestInputTopic<K, V> createInputTopic(final > > > String > > > > topicName); > > > > public final <K, V> TestOutputTopic<K, V> createOutputTopic(final > > > String > > > > topicName); > > > > > > > > KIP-456 can be discarded if this KIP get accepted. > > > > > > > > > > > > Best Regards, > > > > Jukka > > > >