Re: [VOTE] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-23 Thread Jukka Karvanen
Hi All,

Thanks for the votes.

The vote has passed with 3 binding votes and 2 non-binding.

Regards,
Jukka

pe 20. syysk. 2019 klo 20.19 Patrik Kleindl (pklei...@gmail.com) kirjoitti:

> +1 (non-binding)
> Thanks for your efforts
> Best regards
> Patrik
>
> > Am 20.09.2019 um 19:09 schrieb Bill Bejeck :
> >
> > Thanks for the KIP!
> >
> > +1(binding)
> >
> > -Bill
> >
> >> On Fri, Sep 20, 2019 at 10:06 AM Guozhang Wang 
> wrote:
> >>
> >> +1 (binding), Thanks!
> >>
> >> On Fri, Sep 20, 2019 at 9:56 AM Matthias J. Sax 
> >> wrote:
> >>
> >>> Thanks for the KIP Jukka!
> >>>
> >>> +1 (binding)
> >>>
> >>> -Matthias
> >>>
> >>>> On 9/20/19 9:44 AM, Paul Whalen wrote:
> >>>> +1 (non-binding). I haven’t contributed to the discussion but I’ve
> been
> >>> following - it’ll definitely make my team’s life easier.
> >>>>
> >>>>> On Sep 20, 2019, at 11:36 AM, Jukka Karvanen <
> >>> jukka.karva...@jukinimi.com> wrote:
> >>>>>
> >>>>> Hi all,
> >>>>>
> >>>>> I would like to start vote on KIP-470:
> >>>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements
> >>>>>
> >>>>>
> >>>>> Regards,
> >>>>>
> >>>>> Jukka
> >>>
> >>>
> >>
> >> --
> >> -- Guozhang
> >>
>


[VOTE] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-20 Thread Jukka Karvanen
Hi all,

I would like to start vote on KIP-470:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements


Regards,

Jukka


Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-20 Thread Jukka Karvanen
Hi All,

I modified wiki page to replace Serdes with Serializer / Deserializer.

I will start the vote.

Jukka

pe 20. syysk. 2019 klo 4.21 John Roesler (j...@confluent.io) kirjoitti:

> Hey, all,
>
> For what it's worth, I agree with Matthias. While it's true that
> you're likely to have Serdes on hand, if for whatever reason, you
> don't, it's a pain to make one.
>
> Plus, with just requiring Serializer and Deserializer, each component
> is only asking for the minimum thing it needs, rather than passing
> around "kitchen sink" objects.
>
> Likewise, overall, I'm quite happy with this KIP, and I look forward
> to casting my vote.
>
> Thanks, Jukka,
> -John
>
> On Thu, Sep 19, 2019 at 6:03 PM Matthias J. Sax 
> wrote:
> >
> > Personally, I think it would be cleaner to just pass `Deserializer` into
> > `InputTopic` and `Serializer` into `OutputTopic`. Keeping the interface
> > simple is a good argument and we should not have overloads -- but it
> > seems that picking `Serdes` instead of the more straight forward
> > `(De)Serializer` is not "clean". One should only pass what one needs
> IMHO.
> >
> > I also don't think that calling `#serializer()` if necessary is too much
> > boilerplate. At the same time, if you don't have a `Serde` at hand, it
> > would be quite some overhead to create a new Serde that wraps a
> > (de)serializer one has at hand.
> >
> > Note that `ConsumerRecordFactory` also accepts `Serializer` only for the
> > same reason.
> >
> >
> > Beside this minor question, I think you can start a VOTE thread. If we
> > close the vote before Wednesday (cf
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=125307901
> )
> > we might be able to get it into 2.4 release (if we have the bandwidth to
> > review the PR in time...)
> >
> >
> > -Matthias
> >
> >
> >
> > On 9/13/19 10:08 PM, Jukka Karvanen wrote:
> > > Hi,
> > >
> > > In many cases you have already serde for the stream definition. So I
> see
> > > natural to use it also for the tests.
> > > So you can write:
> > >
> > >   public void setup() {
> > > testDriver = new TopologyTestDriver(TestStream.getTopology(),
> > > TestStream.getConfig());
> > > inputTopic = testDriver.createInputTopic(TestStream.INPUT_TOPIC,
> > >  keySerde,  myClassSerde);
> > > outputTopic =
> testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, k
> > > eySerde,  outClassSerde);
> > >   }
> > >
> > > Instead of
> > >
> > >   public void setup() {
> > > testDriver = new TopologyTestDriver(TestStream.getTopology(),
> > > TestStream.getConfig());
> > > inputTopic = testDriver.createInputTopic(TestStream.INPUT_TOPIC,
> > >  keySerde.serializer(),  myClassSerde.serializer());
> > > outputTopic =
> testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, k
> > > eySerde.deserializer(),  outClassSerde.deserializer());
> > >   }
> > >
> > > In original KIP-456 I had variation for Contructors with Serde and
> > > Serializer/Deserializer, but based on the request to keep the
> interface as
> > > simple as possible only version with Serde was kept.
> > > Jukka
> > >
> > >
> > >
> > >
> > >
> > > pe 13. syysk. 2019 klo 19.15 Matthias J. Sax (matth...@confluent.io)
> > > kirjoitti:
> > >
> > >> Maybe one follow up question:
> > >>
> > >> Why do we pass `Serdes` into `createInputTopic` and createOutputTopic`
> > >> -- seems `Serializer` (for input) and `Deserialized` (for output)
> should
> > >> be sufficient?
> > >>
> > >>
> > >> -Matthias
> > >>
> > >> On 9/12/19 4:59 PM, Matthias J. Sax wrote:
> > >>> Thanks for updating the KIP.
> > >>>
> > >>> I agree with John that we (ie, you :)) can start a vote.
> > >>>
> > >>>
> > >>> -Matthias
> > >>>
> > >>> On 9/11/19 9:23 AM, John Roesler wrote:
> > >>>> Thanks for the update, Jukka!
> > >>>>
> > >>>> I'd be in favor of the current proposal. Not sure how the others
> feel.
> > >>>> If people generally feel positive, it might be time to start a vote.
> > >>>>
> > >>>> Thanks,
> > >>>> -John
> > >>>>
> > >>

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-13 Thread Jukka Karvanen
Hi,

In many cases you have already serde for the stream definition. So I see
natural to use it also for the tests.
So you can write:

  public void setup() {
testDriver = new TopologyTestDriver(TestStream.getTopology(),
TestStream.getConfig());
inputTopic = testDriver.createInputTopic(TestStream.INPUT_TOPIC,
 keySerde,  myClassSerde);
outputTopic = testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, k
eySerde,  outClassSerde);
  }

Instead of

  public void setup() {
testDriver = new TopologyTestDriver(TestStream.getTopology(),
TestStream.getConfig());
inputTopic = testDriver.createInputTopic(TestStream.INPUT_TOPIC,
 keySerde.serializer(),  myClassSerde.serializer());
outputTopic = testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, k
eySerde.deserializer(),  outClassSerde.deserializer());
  }

In original KIP-456 I had variation for Contructors with Serde and
Serializer/Deserializer, but based on the request to keep the interface as
simple as possible only version with Serde was kept.
Jukka





pe 13. syysk. 2019 klo 19.15 Matthias J. Sax (matth...@confluent.io)
kirjoitti:

> Maybe one follow up question:
>
> Why do we pass `Serdes` into `createInputTopic` and createOutputTopic`
> -- seems `Serializer` (for input) and `Deserialized` (for output) should
> be sufficient?
>
>
> -Matthias
>
> On 9/12/19 4:59 PM, Matthias J. Sax wrote:
> > Thanks for updating the KIP.
> >
> > I agree with John that we (ie, you :)) can start a vote.
> >
> >
> > -Matthias
> >
> > On 9/11/19 9:23 AM, John Roesler wrote:
> >> Thanks for the update, Jukka!
> >>
> >> I'd be in favor of the current proposal. Not sure how the others feel.
> >> If people generally feel positive, it might be time to start a vote.
> >>
> >> Thanks,
> >> -John
> >>
> >> On Sat, Sep 7, 2019 at 12:40 AM Jukka Karvanen
> >>  wrote:
> >>>
> >>> Hi,
> >>>
> >>> Sorry; I need to rollback right away the one method removal what I was
> >>> proposing.
> >>>
> >>> One constructor with Long restored to TestRecord, which is needed by
> >>> TestInputTopic.
> >>>
> >>> Regards,
> >>> Jukka
> >>>
> >>> la 7. syysk. 2019 klo 8.06 Jukka Karvanen (jukka.karva...@jukinimi.com
> )
> >>> kirjoitti:
> >>>
> >>>> Hi,
> >>>>
> >>>> Let's get back to this after summer break.
> >>>> Thanks Antony to offering help, it might be needed.
> >>>>
> >>>> I modified the KIP based on the feedback to be a mixture of
> variations 4
> >>>> and 5.
> >>>>
> >>>> In TestInputTopic I removed deprecation from one variation with long
> >>>> timestamp and removed totally one version without key.
> >>>> The existing test code with it can be easily migrated to use remaining
> >>>> method adding null key.
> >>>>
> >>>> In TestRecord I removed constructors with Long timestamp from the
> public
> >>>> interface. You can migrate existing code
> >>>> with Long timestamp constructors to use constructors with
> ProducerRecord
> >>>> or ConsumerRecord.
> >>>> There is still Long timestamp(); method like in ProducerRecord /
> >>>> ConsumerRecord.
> >>>>
> >>>> Is this acceptable alternative?
> >>>> What else is needed to conclude the discussion phase and get to
> voting?
> >>>>
> >>>> Regards,
> >>>> Jukka
> >>>>
> >>>> to 5. syysk. 2019 klo 17.17 Antony Stubbs (ant...@confluent.io)
> kirjoitti:
> >>>>
> >>>>> Hi Jukka! I just came across your work - it looks great! I was
> taking a
> >>>>> stab at improving the existing API, but yours already looks great
> and just
> >>>>> about complete! Are you planning on continuing your work and
> submitting a
> >>>>> PR? If you want some help, I'd be happy to jump in.
> >>>>>
> >>>>> Regards,
> >>>>> Antony.
> >>>>>
> >>>>> On Thu, Aug 1, 2019 at 3:42 PM Bill Bejeck 
> wrote:
> >>>>>
> >>>>>> Hi Jukka,
> >>>>>>
> >>>>>> I also think 3, 4, and 5 are all good options.
> >>>>>>
> >>>>>> My personal preference is 4, but I also wouldn't mind going with 5
> if
> >>&

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-06 Thread Jukka Karvanen
Hi,

Sorry; I need to rollback right away the one method removal what I was
proposing.

One constructor with Long restored to TestRecord, which is needed by
TestInputTopic.

Regards,
Jukka

la 7. syysk. 2019 klo 8.06 Jukka Karvanen (jukka.karva...@jukinimi.com)
kirjoitti:

> Hi,
>
> Let's get back to this after summer break.
> Thanks Antony to offering help, it might be needed.
>
> I modified the KIP based on the feedback to be a mixture of variations 4
> and 5.
>
> In TestInputTopic I removed deprecation from one variation with long
> timestamp and removed totally one version without key.
> The existing test code with it can be easily migrated to use remaining
> method adding null key.
>
> In TestRecord I removed constructors with Long timestamp from the public
> interface. You can migrate existing code
> with Long timestamp constructors to use constructors with ProducerRecord
> or ConsumerRecord.
> There is still Long timestamp(); method like in ProducerRecord /
> ConsumerRecord.
>
> Is this acceptable alternative?
> What else is needed to conclude the discussion phase and get to voting?
>
> Regards,
> Jukka
>
> to 5. syysk. 2019 klo 17.17 Antony Stubbs (ant...@confluent.io) kirjoitti:
>
>> Hi Jukka! I just came across your work - it looks great! I was taking a
>> stab at improving the existing API, but yours already looks great and just
>> about complete! Are you planning on continuing your work and submitting a
>> PR? If you want some help, I'd be happy to jump in.
>>
>> Regards,
>> Antony.
>>
>> On Thu, Aug 1, 2019 at 3:42 PM Bill Bejeck  wrote:
>>
>> > Hi Jukka,
>> >
>> > I also think 3, 4, and 5 are all good options.
>> >
>> > My personal preference is 4, but I also wouldn't mind going with 5 if
>> that
>> > is what others want to do.
>> >
>> > Thanks,
>> > Bill
>> >
>> > On Tue, Jul 30, 2019 at 9:31 AM John Roesler  wrote:
>> >
>> > > Hey Jukka,
>> > >
>> > > Sorry for the delay.
>> > >
>> > > For what it's worth, I think 3, 4, and 5 are all good options. I
>> guess my
>> > > own preference is 5.
>> > >
>> > > It seems like the migration pain is a one-time concern vs. having more
>> > > maintainable code for years thereafter.
>> > >
>> > > Thanks,
>> > > -John
>> > >
>> > >
>> > >
>> > > On Tue, Jul 2, 2019 at 4:03 AM Jukka Karvanen <
>> > jukka.karva...@jukinimi.com
>> > > >
>> > > wrote:
>> > >
>> > > > Hi Matthias,
>> > > >
>> > > > Generally I think using Instant and Duration make the test more
>> > readable
>> > > > and that's why I modified KIP based on your suggestion.
>> > > > Now a lot of code use time with long or Long and that make the
>> change
>> > > more
>> > > > complicated.
>> > > >
>> > > > What I tried to say about the migration is the lines without
>> timestamp
>> > or
>> > > > if long timestamp is supported can be migrated mainly with search &
>> > > > recplace:
>> > > >
>> > > >
>> > > >
>> > >
>> >
>> testDriver.pipeInput(recordFactory.create(WordCountLambdaExample.inputTopic,
>> > > > nullKey, "Hello", 1L));
>> > > >
>> > > > ->
>> > > >
>> > > > *inputTopic*.pipeInput(nullKey, "Hello", 1L);
>> > > >
>> > > > If long is not supported as timestamp, the same is not so easy:
>> > > >
>> > > >
>> > > >
>> > >
>> >
>> testDriver.pipeInput(recordFactory.create(WordCountLambdaExample.inputTopic,
>> > > > nullKey, "Hello", 1L));
>> > > >
>> > > > ->
>> > > >
>> > > > *inputTopic1*.pipeInput(nullKey, "Hello", Instant.ofEpochMilli(1L));
>> > > >
>> > > > Also if you need to convert arbitrary long timestamps to proper time
>> > > > Instants, it require you need to understand the logic of the test.
>> So
>> > > > mechanical search & replace is not possible.
>> > > >
>> > > >
>> > > > I see there are several alternatives for long vs Instant / Duration:
>> > > >
>> > > > 1. All times a

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-09-06 Thread Jukka Karvanen
Hi,

Let's get back to this after summer break.
Thanks Antony to offering help, it might be needed.

I modified the KIP based on the feedback to be a mixture of variations 4
and 5.

In TestInputTopic I removed deprecation from one variation with long
timestamp and removed totally one version without key.
The existing test code with it can be easily migrated to use remaining
method adding null key.

In TestRecord I removed constructors with Long timestamp from the public
interface. You can migrate existing code
with Long timestamp constructors to use constructors with ProducerRecord or
ConsumerRecord.
There is still Long timestamp(); method like in ProducerRecord /
ConsumerRecord.

Is this acceptable alternative?
What else is needed to conclude the discussion phase and get to voting?

Regards,
Jukka

to 5. syysk. 2019 klo 17.17 Antony Stubbs (ant...@confluent.io) kirjoitti:

> Hi Jukka! I just came across your work - it looks great! I was taking a
> stab at improving the existing API, but yours already looks great and just
> about complete! Are you planning on continuing your work and submitting a
> PR? If you want some help, I'd be happy to jump in.
>
> Regards,
> Antony.
>
> On Thu, Aug 1, 2019 at 3:42 PM Bill Bejeck  wrote:
>
> > Hi Jukka,
> >
> > I also think 3, 4, and 5 are all good options.
> >
> > My personal preference is 4, but I also wouldn't mind going with 5 if
> that
> > is what others want to do.
> >
> > Thanks,
> > Bill
> >
> > On Tue, Jul 30, 2019 at 9:31 AM John Roesler  wrote:
> >
> > > Hey Jukka,
> > >
> > > Sorry for the delay.
> > >
> > > For what it's worth, I think 3, 4, and 5 are all good options. I guess
> my
> > > own preference is 5.
> > >
> > > It seems like the migration pain is a one-time concern vs. having more
> > > maintainable code for years thereafter.
> > >
> > > Thanks,
> > > -John
> > >
> > >
> > >
> > > On Tue, Jul 2, 2019 at 4:03 AM Jukka Karvanen <
> > jukka.karva...@jukinimi.com
> > > >
> > > wrote:
> > >
> > > > Hi Matthias,
> > > >
> > > > Generally I think using Instant and Duration make the test more
> > readable
> > > > and that's why I modified KIP based on your suggestion.
> > > > Now a lot of code use time with long or Long and that make the change
> > > more
> > > > complicated.
> > > >
> > > > What I tried to say about the migration is the lines without
> timestamp
> > or
> > > > if long timestamp is supported can be migrated mainly with search &
> > > > recplace:
> > > >
> > > >
> > > >
> > >
> >
> testDriver.pipeInput(recordFactory.create(WordCountLambdaExample.inputTopic,
> > > > nullKey, "Hello", 1L));
> > > >
> > > > ->
> > > >
> > > > *inputTopic*.pipeInput(nullKey, "Hello", 1L);
> > > >
> > > > If long is not supported as timestamp, the same is not so easy:
> > > >
> > > >
> > > >
> > >
> >
> testDriver.pipeInput(recordFactory.create(WordCountLambdaExample.inputTopic,
> > > > nullKey, "Hello", 1L));
> > > >
> > > > ->
> > > >
> > > > *inputTopic1*.pipeInput(nullKey, "Hello", Instant.ofEpochMilli(1L));
> > > >
> > > > Also if you need to convert arbitrary long timestamps to proper time
> > > > Instants, it require you need to understand the logic of the test. So
> > > > mechanical search & replace is not possible.
> > > >
> > > >
> > > > I see there are several alternatives for long vs Instant / Duration:
> > > >
> > > > 1. All times as long/Long like in this version:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=119550011
> > > >
> > > > (startTimestampMs, autoAdvanceMs as parameter of  createInputTopic
> > > > instead of configureTiming)
> > > >
> > > > 2. Auto timestamping configured with Instant and Duration, pipeInput
> > > > and TestRecord with long:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=120722523
> > > >
> > > >
> > > > 3. (CURRENT) Auto timestamping configured with Instant and Duration,
> > >

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-07-02 Thread Jukka Karvanen
Hi Matthias,

Generally I think using Instant and Duration make the test more readable
and that's why I modified KIP based on your suggestion.
Now a lot of code use time with long or Long and that make the change more
complicated.

What I tried to say about the migration is the lines without timestamp or
if long timestamp is supported can be migrated mainly with search &
recplace:

testDriver.pipeInput(recordFactory.create(WordCountLambdaExample.inputTopic,
nullKey, "Hello", 1L));

->

*inputTopic*.pipeInput(nullKey, "Hello", 1L);

If long is not supported as timestamp, the same is not so easy:

testDriver.pipeInput(recordFactory.create(WordCountLambdaExample.inputTopic,
nullKey, "Hello", 1L));

->

*inputTopic1*.pipeInput(nullKey, "Hello", Instant.ofEpochMilli(1L));

Also if you need to convert arbitrary long timestamps to proper time
Instants, it require you need to understand the logic of the test. So
mechanical search & replace is not possible.


I see there are several alternatives for long vs Instant / Duration:

1. All times as long/Long like in this version:

https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=119550011

(startTimestampMs, autoAdvanceMs as parameter of  createInputTopic
instead of configureTiming)

2. Auto timestamping configured with Instant and Duration, pipeInput
and TestRecord with long:

https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=120722523


3. (CURRENT) Auto timestamping configured with Instant and Duration,
pipeInput and TestRecord with Instant, version with long deprecated:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements


4. Auto timestamping configured with Instant and Duration, pipeInput
and TestRecord with Instant and long parallel (with long not
deprecated):

5. Auto timestamping configured with Instant and Duration, pipeInput
and TestRecord with Instant only

I hope to get feedback.

My own preference currently is alternative 3. or 4.


If somebody want to test, there is a implementation of this version
available in Github:

https://github.com/jukkakarvanen/kafka-streams-test-topics

which can be used directly from public Maven repository:


com.github.jukkakarvanen
kafka-streams-test-topics
0.0.1-beta3
test


Also is this approach in KIP-470 preferred over KIP-456, so can we close it:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver

Jukka

.


pe 28. kesäk. 2019 klo 1.10 Matthias J. Sax (matth...@confluent.io)
kirjoitti:

> Thanks Jukka!
>
> The idea to use `Instant/Duration` was a proposal. If we think it's not
> a good one, we could still stay with `long`. Because `ProducerRecord`
> and `ConsumerRecord` are both based on `long,` it might make sense to
> keep `long`?
>
> > The result of converting millis to Instant directly generates
> >> rather non readable test code and changing from long to Instant
> correctly
> >> require understand what is the case it is testing.
>
> This might be a good indicator the using `Instant/Duration` might not be
> a good idea.
>
> Would be nice to get feedback from others.
>
> About adding new methods that we deprecate immediately: I don't think we
> should do this. IMHO, there are two kind of users, one that immediately
> rewrite their code to move off deprecated methods. Those won't use the
> new+deprecated ones anyway. Other uses migrate their code slowly and
> would just not rewrite their code at all, and thus also not use the
> new+deprecated methods.
>
> > Checking my own tests I was able to migrate the most of my code with
> > search without further thinking about the logic to this new
> > approach. The result of converting millis to Instant directly generates
> > rather non readable test code and changing from long to Instant correctly
> > require understand what is the case it is testing.
>
> Not sure if I can follow here. You first say, you could easily migrate
> your code, but than you say it was not easily possible? Can you clarify
> your experience upgrading your test code?
>
>
> -Matthias
>
>
> On 6/27/19 12:21 AM, Jukka Karvanen wrote:
> > Hi,
> >
> >>> (4) Should we switch from `long` for timestamps to `Instant` and
> > `Duration` ?
> >> This version startTimestamp is Instant and autoAdvance Duration in
> > Initialization and with manual configured collection pipe methods.
> >> Now timestamp of TestRecord is still Long and similarly single record
> > pipeInput still has long as parameter.
> >> Should these also converted to to Instant type?
> >> Should there be both long and Instant parallel?
> 

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-27 Thread Jukka Karvanen
Hi,

>>(4) Should we switch from `long` for timestamps to `Instant` and
`Duration` ?
>This version startTimestamp is Instant and autoAdvance Duration in
Initialization and with manual configured collection pipe methods.
>Now timestamp of TestRecord is still Long and similarly single record
pipeInput still has long as parameter.
>Should these also converted to to Instant type?
>Should there be both long and Instant parallel?
>I expect there are existing test codebase which would be easier to migrate
if long could be still used.
Now added Instant version to TestRecord and pipeInput method.

Checking my own tests I was able to migrate the most of my code with
search without further thinking about the logic to this new
approach. The result of converting millis to Instant directly generates
rather non readable test code and changing from long to Instant correctly
require understand what is the case it is testing.

That is why version with long left still as deprecated for easier migration
for existing test.
Also TopologyTestDriver constructor and advanceWallClockTime  method
modified with same approach.

Jukka


ma 24. kesäk. 2019 klo 16.47 Bill Bejeck (bbej...@gmail.com) kirjoitti:

> Hi Jukka
>
> > These topic objects are only interfacing TopologyTestDriver, not
> affecting
> > the internal functionality of it. In my plan the internal data structures
> > are using those Producer/ConsumerRecords as earlier. That way I don't see
> > how those could be affected.
>
> I mistakenly thought the KIP was proposing to completely remove
> Producer/ConsumerRecords in favor of TestRecord.  But taking another quick
> look I can see the plan for using the TestRecord objects.  Thanks for the
> clarification.
>
> -Bill
>
> On Sat, Jun 22, 2019 at 2:26 AM Jukka Karvanen <
> jukka.karva...@jukinimi.com>
> wrote:
>
> > Hi All,
> > Hi Matthias,
> >
> > >(1) It's a little confusing that you list all method (existing, proposed
> > >to deprecate, and new one) of `TopologyTestDriver` in the KIP. Maybe
> > >only list the ones you propose to deprecate and the new ones you want to
> > >add?
> >
> > Ok. Unmodified methods removed.
> >
> > >(2) `TopologyTestDriver#createInputTopic`: might it be worth to add
> > >overload to initialize the timetamp and auto-advance feature directly?
> > >Otherwise, uses always need to call `configureTiming` as an extra call?
> >
> > Added with Instant and Duration parameters.
> >
> > >(3) `TestInputTopic#configureTiming()`: maybe rename to
> > `reconfigureTiming()` ?
> >
> > I removed this method when we have now initialization in constructor.
> > You can recreate TestInputTopic if needing to reconfigure timing.
> >
> >
> > >(4) Should we switch from `long` for timestamps to `Instant` and
> > `Duration` ?
> > This version startTimestamp is Instant and autoAdvance Duration in
> > Initialization and with manual configured collection pipe methods.
> > Now timestamp of TestRecord is still Long and similarly single record
> > pipeInput still has long as parameter.
> > Should these also converted to to Instant type?
> > Should there be both long and Instant parallel?
> > I expect there are existing test codebase which would be easier to
> migrate
> > if long could be still used.
> >
> > Also should advanceWallClockTime  in TopologyTestDriver changed(or added
> > alternative) for Duration parameter.
> >
> >
> > > (5) Why do we have redundant getters? Or set with `getX()` and one
> > set without `get`-prefix?
> >
> > The methods without get-prefix are for compatibility with
> ProducerRecord /
> > ConsumerRecord and I expect would make migration to TestRecord easier.
> > Standard getters in TestRecord enable writing test ignoring selected
> fields
> > with hamcrest like this:
> >
> > assertThat(outputTopic.readRecord(), allOf(
> > hasProperty("key", equalTo(1L)),
> > hasProperty("value", equalTo("Hello")),
> > hasProperty("headers", equalTo(headers;
> >
> >
> > That's why I have currently both methods.
> >
> > Jukka
> >
> >
> > pe 21. kesäk. 2019 klo 22.20 Matthias J. Sax (matth...@confluent.io)
> > kirjoitti:
> >
> > > Thanks for the KIP. The idea to add InputTopic and OutputTopic
> > > abstractions is really neat!
> > >
> > >
> > > Couple of minor comment:
> > >
> > > (1) It's a little confusing that you list all method (existing,
> proposed
> > > to deprecate, an

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-22 Thread Jukka Karvanen
Hi All,
Hi Matthias,

>(1) It's a little confusing that you list all method (existing, proposed
>to deprecate, and new one) of `TopologyTestDriver` in the KIP. Maybe
>only list the ones you propose to deprecate and the new ones you want to
>add?

Ok. Unmodified methods removed.

>(2) `TopologyTestDriver#createInputTopic`: might it be worth to add
>overload to initialize the timetamp and auto-advance feature directly?
>Otherwise, uses always need to call `configureTiming` as an extra call?

Added with Instant and Duration parameters.

>(3) `TestInputTopic#configureTiming()`: maybe rename to
`reconfigureTiming()` ?

I removed this method when we have now initialization in constructor.
You can recreate TestInputTopic if needing to reconfigure timing.


>(4) Should we switch from `long` for timestamps to `Instant` and
`Duration` ?
This version startTimestamp is Instant and autoAdvance Duration in
Initialization and with manual configured collection pipe methods.
Now timestamp of TestRecord is still Long and similarly single record
pipeInput still has long as parameter.
Should these also converted to to Instant type?
Should there be both long and Instant parallel?
I expect there are existing test codebase which would be easier to migrate
if long could be still used.

Also should advanceWallClockTime  in TopologyTestDriver changed(or added
alternative) for Duration parameter.


> (5) Why do we have redundant getters? Or set with `getX()` and one
set without `get`-prefix?

The methods without get-prefix are for compatibility with  ProducerRecord /
ConsumerRecord and I expect would make migration to TestRecord easier.
Standard getters in TestRecord enable writing test ignoring selected fields
with hamcrest like this:

assertThat(outputTopic.readRecord(), allOf(
hasProperty("key", equalTo(1L)),
hasProperty("value", equalTo("Hello")),
hasProperty("headers", equalTo(headers;


That's why I have currently both methods.

Jukka


pe 21. kesäk. 2019 klo 22.20 Matthias J. Sax (matth...@confluent.io)
kirjoitti:

> Thanks for the KIP. The idea to add InputTopic and OutputTopic
> abstractions is really neat!
>
>
> Couple of minor comment:
>
> (1) It's a little confusing that you list all method (existing, proposed
> to deprecate, and new one) of `TopologyTestDriver` in the KIP. Maybe
> only list the ones you propose to deprecate and the new ones you want to
> add?
>
> (Or mark all existing methods clearly -- atm, I need to got back to the
> code to read the KIP and to extract what changes are proposed).
>
>
> (2) `TopologyTestDriver#createInputTopic`: might it be worth to add
> overload to initialize the timetamp and auto-advance feature directly?
> Otherwise, uses always need to call `configureTiming` as an extra call?
>
>
> (3) `TestInputTopic#configureTiming()`: maybe rename to
> `reconfigureTiming()` ?
>
>
> (4) Should we switch from `long` for timestamps to `Instant` and
> `Duration` ?
>
>
> (5) Why do we have redundant getters? Or set with `getX()` and one set
> without `get`-prefix?
>
>
>
> -Matthias
>
>
>
>
> On 6/21/19 10:57 AM, Bill Bejeck wrote:
> > Jukka,
> >
> > Thanks for the KIP. I like the changes overall.
> > One thing I wanted to confirm, and this may be me being paranoid, but
> will
> > the changes for input/output topic affect how the TopologyTestDriver
> works
> > with internal topics when there are sub-topologies created?
> >
> > On Fri, Jun 21, 2019 at 12:05 PM Guozhang Wang 
> wrote:
> >
> >> 1) Got it, could you list this class along with all its functions in the
> >> proposed public APIs as well?
> >>
> >> 2) Ack, thanks!
> >>
> >> On Thu, Jun 20, 2019 at 11:27 PM Jukka Karvanen <
> >> jukka.karva...@jukinimi.com>
> >> wrote:
> >>
> >>> Hi  Guozhang,
> >>>
> >>> 1) This TestRecord is new class in my proposal. So it is a simplified
> >>> version of ProducerRecord and ConsumerRecord containing only the fields
> >>> needed to test record content.
> >>>
> >>> 2)
> >>> public final  TestInputTopic createInputTopic(final String
> >>> topicName, final Serde keySerde, final Serde valueSerde);
> >>> public final  TestOutputTopic createOutputTopic(final
> String
> >>> topicName, final Serde keySerde, final Serde valueSerde);
> >>> The purpose is to create separate object for each input and output
> topic
> >>> you are using. The topic name is given to createInput/OutputTopic when
> >>> initialize topic object.
> >>>
> >>> For example:
> >>>

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Jukka Karvanen
Hi Bill,

These topic objects are only interfacing TopologyTestDriver, not affecting
the internal functionality of it. In my plan the internal data structures
are using those Producer/ConsumerRecords as earlier. That way I don't see
how those could be affected.

Jukka



On Fri, 21 Jun 2019, 20:57 Bill Bejeck,  wrote:

> Jukka,
>
> Thanks for the KIP. I like the changes overall.
> One thing I wanted to confirm, and this may be me being paranoid, but will
> the changes for input/output topic affect how the TopologyTestDriver works
> with internal topics when there are sub-topologies created?
>
> On Fri, Jun 21, 2019 at 12:05 PM Guozhang Wang  wrote:
>
> > 1) Got it, could you list this class along with all its functions in the
> > proposed public APIs as well?
> >
> > 2) Ack, thanks!
> >
> > On Thu, Jun 20, 2019 at 11:27 PM Jukka Karvanen <
> > jukka.karva...@jukinimi.com>
> > wrote:
> >
> > > Hi  Guozhang,
> > >
> > > 1) This TestRecord is new class in my proposal. So it is a simplified
> > > version of ProducerRecord and ConsumerRecord containing only the fields
> > > needed to test record content.
> > >
> > > 2)
> > > public final  TestInputTopic createInputTopic(final String
> > > topicName, final Serde keySerde, final Serde valueSerde);
> > > public final  TestOutputTopic createOutputTopic(final
> String
> > > topicName, final Serde keySerde, final Serde valueSerde);
> > > The purpose is to create separate object for each input and output
> topic
> > > you are using. The topic name is given to createInput/OutputTopic when
> > > initialize topic object.
> > >
> > > For example:
> > >
> > > final TestInputTopic inputTopic1 =
> > > testDriver.createInputTopic( INPUT_TOPIC, longSerde, stringSerde);
> > > final TestInputTopic inputTopic2 =
> > > testDriver.createInputTopic( INPUT_TOPIC_MAP, longSerde, stringSerde);
> > > final TestOutputTopic outputTopic1 =
> > > testDriver.createOutputTopic(OUTPUT_TOPIC, longSerde, stringSerde);
> > > final TestOutputTopic outputTopic2 =
> > > testDriver.createOutputTopic(OUTPUT_TOPIC_MAP, stringSerde,
> > > longSerde);
> > > inputTopic1.pipeInput(1L, "Hello");
> > > assertThat(outputTopic1.readKeyValue(), equalTo(new KeyValue<>(1L,
> > > "Hello")));
> > > assertThat(outputTopic2.readKeyValue(), equalTo(new KeyValue<>("Hello",
> > > 1L)));
> > > inputTopic2.pipeInput(1L, "Hello");
> > >
> > >
> > > Jukka
> > >
> > > to 20. kesäk. 2019 klo 23.52 Guozhang Wang (wangg...@gmail.com)
> > kirjoitti:
> > >
> > > > Hello Jukka,
> > > >
> > > > Thanks for writing the KIP, I have a couple of quick questions:
> > > >
> > > > 1) Is "TestRecord" an existing class that you propose to piggy-back
> on?
> > > > Right now we have a scala TestRecord case class but I doubt that was
> > your
> > > > proposal, or are you proposing to add a new Java class?
> > > >
> > > > 2) Would the new API only allow a single input / output topic with
> > > > `createInput/OutputTopic`? If not, when we call pipeInput how to
> > > determine
> > > > which topic this record should be pipe to?
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Mon, Jun 17, 2019 at 1:34 PM John Roesler 
> > wrote:
> > > >
> > > > > Woah, I wasn't aware of that Hamcrest test style. Awesome!
> > > > >
> > > > > Thanks for the updates. I look forward to hearing what others
> think.
> > > > >
> > > > > -John
> > > > >
> > > > > On Mon, Jun 17, 2019 at 4:12 AM Jukka Karvanen
> > > > >  wrote:
> > > > > >
> > > > > > Wiki page updated:
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements
> > > > > >
> > > > > >
> > > > > > ClientRecord removed and replaced with TestRecord in method
> calls.
> > > > > > TestRecordFactory removed (time tracking functionality to be
> > included
> > > > to
> > > > > > TestInputTopic)
> > > > > > OutputVerifier deprecated
> >

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Jukka Karvanen
Hi,

TestRecord is already included in wiki page. It has mainly constructors and
methods to access data fields.

Jukka

On Fri, 21 Jun 2019, 19:05 Guozhang Wang,  wrote:

> 1) Got it, could you list this class along with all its functions in the
> proposed public APIs as well?
>
> 2) Ack, thanks!
>
> On Thu, Jun 20, 2019 at 11:27 PM Jukka Karvanen <
> jukka.karva...@jukinimi.com>
> wrote:
>
> > Hi  Guozhang,
> >
> > 1) This TestRecord is new class in my proposal. So it is a simplified
> > version of ProducerRecord and ConsumerRecord containing only the fields
> > needed to test record content.
> >
> > 2)
> > public final  TestInputTopic createInputTopic(final String
> > topicName, final Serde keySerde, final Serde valueSerde);
> > public final  TestOutputTopic createOutputTopic(final String
> > topicName, final Serde keySerde, final Serde valueSerde);
> > The purpose is to create separate object for each input and output topic
> > you are using. The topic name is given to createInput/OutputTopic when
> > initialize topic object.
> >
> > For example:
> >
> > final TestInputTopic inputTopic1 =
> > testDriver.createInputTopic( INPUT_TOPIC, longSerde, stringSerde);
> > final TestInputTopic inputTopic2 =
> > testDriver.createInputTopic( INPUT_TOPIC_MAP, longSerde, stringSerde);
> > final TestOutputTopic outputTopic1 =
> > testDriver.createOutputTopic(OUTPUT_TOPIC, longSerde, stringSerde);
> > final TestOutputTopic outputTopic2 =
> > testDriver.createOutputTopic(OUTPUT_TOPIC_MAP, stringSerde,
> > longSerde);
> > inputTopic1.pipeInput(1L, "Hello");
> > assertThat(outputTopic1.readKeyValue(), equalTo(new KeyValue<>(1L,
> > "Hello")));
> > assertThat(outputTopic2.readKeyValue(), equalTo(new KeyValue<>("Hello",
> > 1L)));
> > inputTopic2.pipeInput(1L, "Hello");
> >
> >
> > Jukka
> >
> > to 20. kesäk. 2019 klo 23.52 Guozhang Wang (wangg...@gmail.com)
> kirjoitti:
> >
> > > Hello Jukka,
> > >
> > > Thanks for writing the KIP, I have a couple of quick questions:
> > >
> > > 1) Is "TestRecord" an existing class that you propose to piggy-back on?
> > > Right now we have a scala TestRecord case class but I doubt that was
> your
> > > proposal, or are you proposing to add a new Java class?
> > >
> > > 2) Would the new API only allow a single input / output topic with
> > > `createInput/OutputTopic`? If not, when we call pipeInput how to
> > determine
> > > which topic this record should be pipe to?
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, Jun 17, 2019 at 1:34 PM John Roesler 
> wrote:
> > >
> > > > Woah, I wasn't aware of that Hamcrest test style. Awesome!
> > > >
> > > > Thanks for the updates. I look forward to hearing what others think.
> > > >
> > > > -John
> > > >
> > > > On Mon, Jun 17, 2019 at 4:12 AM Jukka Karvanen
> > > >  wrote:
> > > > >
> > > > > Wiki page updated:
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements
> > > > >
> > > > >
> > > > > ClientRecord removed and replaced with TestRecord in method calls.
> > > > > TestRecordFactory removed (time tracking functionality to be
> included
> > > to
> > > > > TestInputTopic)
> > > > > OutputVerifier deprecated
> > > > > TestRecord topic removed and getters added
> > > > >
> > > > > Getters in TestRecord enable writing test ignoring selected fields
> > with
> > > > > hamcrest like this:
> > > > >
> > > > > assertThat(outputTopic.readRecord(), allOf(
> > > > > hasProperty("key", equalTo(1L)),
> > > > > hasProperty("value", equalTo("Hello")),
> > > > > hasProperty("headers", equalTo(headers;
> > > > >
> > > > >
> > > > > Jukka
> > > > >
> > > > > la 15. kesäk. 2019 klo 1.10 John Roesler (j...@confluent.io)
> > > kirjoitti:
> > > > >
> > > > > > Sounds good. Thanks as always for considering my feedback!
> > > > > > -John
> > > > > >
> > > >

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Jukka Karvanen
Hi  Guozhang,

1) This TestRecord is new class in my proposal. So it is a simplified
version of ProducerRecord and ConsumerRecord containing only the fields
needed to test record content.

2)
public final  TestInputTopic createInputTopic(final String
topicName, final Serde keySerde, final Serde valueSerde);
public final  TestOutputTopic createOutputTopic(final String
topicName, final Serde keySerde, final Serde valueSerde);
The purpose is to create separate object for each input and output topic
you are using. The topic name is given to createInput/OutputTopic when
initialize topic object.

For example:

final TestInputTopic inputTopic1 =
testDriver.createInputTopic( INPUT_TOPIC, longSerde, stringSerde);
final TestInputTopic inputTopic2 =
testDriver.createInputTopic( INPUT_TOPIC_MAP, longSerde, stringSerde);
final TestOutputTopic outputTopic1 =
testDriver.createOutputTopic(OUTPUT_TOPIC, longSerde, stringSerde);
final TestOutputTopic outputTopic2 =
testDriver.createOutputTopic(OUTPUT_TOPIC_MAP, stringSerde,
longSerde);
inputTopic1.pipeInput(1L, "Hello");
assertThat(outputTopic1.readKeyValue(), equalTo(new KeyValue<>(1L, "Hello")));
assertThat(outputTopic2.readKeyValue(), equalTo(new KeyValue<>("Hello", 1L)));
inputTopic2.pipeInput(1L, "Hello");


Jukka

to 20. kesäk. 2019 klo 23.52 Guozhang Wang (wangg...@gmail.com) kirjoitti:

> Hello Jukka,
>
> Thanks for writing the KIP, I have a couple of quick questions:
>
> 1) Is "TestRecord" an existing class that you propose to piggy-back on?
> Right now we have a scala TestRecord case class but I doubt that was your
> proposal, or are you proposing to add a new Java class?
>
> 2) Would the new API only allow a single input / output topic with
> `createInput/OutputTopic`? If not, when we call pipeInput how to determine
> which topic this record should be pipe to?
>
>
> Guozhang
>
> On Mon, Jun 17, 2019 at 1:34 PM John Roesler  wrote:
>
> > Woah, I wasn't aware of that Hamcrest test style. Awesome!
> >
> > Thanks for the updates. I look forward to hearing what others think.
> >
> > -John
> >
> > On Mon, Jun 17, 2019 at 4:12 AM Jukka Karvanen
> >  wrote:
> > >
> > > Wiki page updated:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements
> > >
> > >
> > > ClientRecord removed and replaced with TestRecord in method calls.
> > > TestRecordFactory removed (time tracking functionality to be included
> to
> > > TestInputTopic)
> > > OutputVerifier deprecated
> > > TestRecord topic removed and getters added
> > >
> > > Getters in TestRecord enable writing test ignoring selected fields with
> > > hamcrest like this:
> > >
> > > assertThat(outputTopic.readRecord(), allOf(
> > > hasProperty("key", equalTo(1L)),
> > > hasProperty("value", equalTo("Hello")),
> > > hasProperty("headers", equalTo(headers;
> > >
> > >
> > > Jukka
> > >
> > > la 15. kesäk. 2019 klo 1.10 John Roesler (j...@confluent.io)
> kirjoitti:
> > >
> > > > Sounds good. Thanks as always for considering my feedback!
> > > > -John
> > > >
> > > > On Fri, Jun 14, 2019 at 12:12 PM Jukka Karvanen
> > > >  wrote:
> > > > >
> > > > > 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, 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 inputValues = Arrays.asList(
> > > > > "Apache Kafka Streams Example"

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-17 Thread Jukka Karvanen
Wiki page updated:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements


ClientRecord removed and replaced with TestRecord in method calls.
TestRecordFactory removed (time tracking functionality to be included to
TestInputTopic)
OutputVerifier deprecated
TestRecord topic removed and getters added

Getters in TestRecord enable writing test ignoring selected fields with
hamcrest like this:

assertThat(outputTopic.readRecord(), allOf(
hasProperty("key", equalTo(1L)),
hasProperty("value", equalTo("Hello")),
hasProperty("headers", equalTo(headers;


Jukka

la 15. kesäk. 2019 klo 1.10 John Roesler (j...@confluent.io) kirjoitti:

> Sounds good. Thanks as always for considering my feedback!
> -John
>
> On Fri, Jun 14, 2019 at 12:12 PM Jukka Karvanen
>  wrote:
> >
> > 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, 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 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 i

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-14 Thread Jukka Karvanen
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, 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 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
>  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.
> &

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-14 Thread Jukka Karvanen
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
>  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  TestInputTopic createInputTopic(final
> String
> > topicName, final Serde keySerde, final Serde valueSerde);
> > public final  TestOutputTopic createOutputTopic(final
> String
> > topicName, final Serde keySerde, final Serde 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  TestInputTopic createInputTopic(final
> String
> > topicName);
> > public final  TestOutputTopic createOutputTopic(final
> String
> > topicName);
> >
> > KIP-456 can be discarded if this KIP get accepted.
> >
> >
> > Best Regards,
> > Jukka
>


Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-20 Thread Jukka Karvanen
Hi All,

Inspired by the discussion in this thread, there is a new 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 can be discarded if KIP-470 get accepted.

Even this KIP might be rejected, migrating from this classes to KIP-470 is
rather straighforward.
There are addon package which can be used with any Kafka version
>=1.1.0.before these are included to release.

See info:
https://github.com/jukkakarvanen/kafka-streams-test-topics

Maven package:
https://mvnrepository.com/artifact/com.github.jukkakarvanen/kafka-streams-test-topics


Best Regards,
Jukka

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

> Hi Jukka
> Sorry, that was mostly what I had in mind, I didn't have enough time to
> look through the KIP.
>
> My question was also if this handling of topics wouldn't make more sense
> even outside the TTD, for the general API.
>
> regards
> Patrik
>
> On Thu, 9 May 2019 at 14:43, Jukka Karvanen 
> wrote:
>
> > 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 inputTopic = new TestInputTopic > String>(testDriver, INPUT_TOPIC, new Serdes.StringSerde(), new
> > Serdes.StringSerde());
> >
> > So if TopologyTestDriver modified, this could be
> >
> > TestInputTopic inputTopic =
> > testDriver.getInputTopic(INPUT_TOPIC, new Serdes.StringSerde(), new
> > Serdes.StringSerde());
> >
> > or preferrable if serders can be found:
> >
> > TestInputTopic 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.
> >

[DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-05-20 Thread Jukka Karvanen
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

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  TestInputTopic createInputTopic(final String
topicName, final Serde keySerde, final Serde valueSerde);
public final  TestOutputTopic createOutputTopic(final String
topicName, final Serde keySerde, final Serde 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  TestInputTopic createInputTopic(final String
topicName);
public final  TestOutputTopic createOutputTopic(final String
topicName);

KIP-456 can be discarded if this KIP get accepted.


Best Regards,
Jukka


Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-09 Thread Jukka Karvanen
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 inputTopic = new TestInputTopic(testDriver, INPUT_TOPIC, new Serdes.StringSerde(), new
Serdes.StringSerde());

So if TopologyTestDriver modified, this could be

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

or preferrable if serders can be found:

TestInputTopic 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 
> 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

Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-09 Thread Jukka Karvanen
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 

Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-08 Thread Jukka Karvanen
Hi,

This KIP is relating to add classes to kafka-stream-test-utils.
My initial plan was to create new project for these helper classes before I
decided to contribute directly to Apache Kafka with this KIP.
I have one streams project where I have used these classes, that is why I
planned to release this as separate "early access" package
before the Kafka release including this is publicly available. This would
work at the same time "a historically-compatible version of the library".

I try to summarize the open topics after your last email:

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?

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
- changing pipeInput to take in ProducerRecord, but not providing easy way
to contruct those like ConsumerRecordFactory
- 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

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


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.

4. IllegalArgumentException("topic doesn't exist")
Is this worth separate ticket?

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?

Jukka


ti 7. toukok. 2019 klo 22.02 John Roesler (j...@confluent.io) kirjoitti:

> Thanks for the responses, Jukka!
>
> Thanks for the reference to the javadocs in your library, but I
> actually meant they should be part of the KIP document.
>
> As a general comment, I did get that your intent is to smooth over
> some rough spots with the current testing library, and that many of
> your API/behavior decisions are just reflecting the underlying TTD.
> What I'm wondering is whether we can take the opportunity to make a
> bigger change for an even smoother usability experience.
>
> Regarding the publishing artifact, did you mean that you're proposing
> to add a separate build artifact to the Apache Kafka project, or just
> that you plan to host a historically-compatible version of the library
> on your own? My initial reaction is that we should just include this
> stuff in the test-utils package starting at whatever version. This
> gives us greater latitude to modify the underlying layers in service
> of this KIP. Plus, it's nicer as a user to just have one test-support
> artifact to manage.
>
> Specific responses below:
> > >InputTopic:
> > >
> > >1. Have you considered adding a ProducerRecord input method to the input
> > >topic? This might be unnecessary, given the overloads you provide. I'm
> > >wondering if it decreases the domain mismatch between TopologyTestDriver
> > >and KafkaStreams, though, since in production code, you send input to
> the
> > >app as a ProducerRecord, via a topic. Also, this might let you drop
> some of
> > >the less mainstream overloads, like the ones for headers.
> >
> > Ok, so not same methods as in TopologyTestDriver:
> > pipeInput(ConsumerRecord consumerRecord);
> > pipeInput(List> records);
> > but instead:
> > pipeInput(ProducerRecord  record);
> > pipeInput(List< ProducerRecord  > records);
> > Which would convert ProductRecords to ConsumerRecords before piping to
> TTD
> >
> > and Drop these methods
> > void pipeInput(K key, V value, Headers headers);
> > void pipeInput(K key, V value, Headers headers, long timestampMs);
>
> Yes, this is essentially what I was saying.
>
> >
> > In this case you would not able to create those with
> ConsumerRecordFactory,
> > but needing to create  ProducerRecord   objects directly.one by one 

Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-07 Thread Jukka Karvanen
per record, or to be able to verify the
> result, given specific timestamps as input. Did you consider a
> KeyValueTimestamp value type instead? Alternatively, if you like the
> ProducerRecord approach, above, you could lean on that instead.
>
> 3. I wasn't clear on the semantics of the constructors that take a start
> timestamp, but no advance time. I also wasn't clear on the semantics when
> the constructor specifies start/advance, but then we also call the input
> methods that specify timestamps, or start/advance timestamps. Also related,
> what's the "default" timestamp, if no start is specified, "zero" or "now"
> both seem reasonable. Similar with the advance, "1ms" and "0ms" both seem
> reasonable defaults.
>
> OutputTopic:
>
> 4. Tentatively, ProducerRecord seems like a strange output type, since as a
> user, I'm "consuming" the results. How about using ConsumerRecord instead?
>
> 5. We have methods above for producing with specific timestamps, but none
> for observing the timestamps. How can we strengthen the symmetry?
>
> 6. (just a comment) I like the readToMap method. Thanks!
>
> 7. I know it clashes somewhat with the Kafka semantics, but I'm a little
> concerned about null semantics in the output topic. In particular,
> "readValue()" returning null ambiguously indicates both "there is no value"
> and "there is a null value present". Can we consider throwing a
> NoSuchElementException when you attempt to read, but there is nothing
> there?
>
> 7.5. This would necessitate adding a boolean method to query the presence
> of output records, which can be a handy way to cap off a test:
> `assertFalse(outputTopic.hasRecords(),
> outputTopic.readKeyValuesToList().toString())` would fail and print out the
> remaining records if there are any.
>
> General:
>
> 8. Can the TTD, input, and output topics implement AutoCloseable, to
> facilitate try-with-resources in tests?
>
> Example:
> try (driver = new TTD(), input = new TestInputTopic(), output = new
> TestOutputTopic() ) {
>  ...
> } // ensures everything is closed
>
> 7. Should we add some assertion that all the output is consumed when you
> call testDriver.close() ?
>
> 8. Should we consider deprecating (some or all) of the overlapping
> mechanisms in TopologyTestDriver? It seems like this might improve
> usability by reducing ambiguity.
>
> 9. Can you give us some idea of the javadoc for each of the new methods
> you're proposing? The documentation is also part of the public API, and it
> also helps us understand the semantics of the operations you're proposing.
>
> That's all! Thanks so much for this awesome proposal,
> -John
>
> On Mon, May 6, 2019 at 6:58 AM Jukka Karvanen  >
> wrote:
>
> > Hi,
> >
> > Now everything what I planned to add including tests, samples and
> document
> > changes are in my branch:
> >
> >
> https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics
> >
> >
> > So I can create PR as soon as this KIP is getting green light to proceed.
> >
> > Jukka
> >
> > la 4. toukok. 2019 klo 9.05 Jukka Karvanen (jukka.karva...@jukinimi.com)
> > kirjoitti:
> >
> > > Hi,
> > >
> > > New TestInputTopic and TestOutputTopic included to Developer guide
> > testing
> > > page as alternative,
> > > The old way with ConsumerRecordFactory and OutputVerifier is not
> removed.
> > >
> > > You can see the proposal here in my branch:
> > >
> > >
> >
> https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics
> > >
> > >
> > > I can create Work In progress pull request if that make commenting
> > > proposal easier.
> > > Still planning to add full coverage unit test and sample
> > WordCountDemoTest to
> > >
> >
> streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount,
> > > if this KIP is accepted.
> > >
> > > Jukka
> > >
> > >
> > > ti 30. huhtik. 2019 klo 13.59 Matthias J. Sax (matth...@confluent.io)
> > > kirjoitti:
> > >
> > >> KIP-451 was discarded in favor this this KIP. So it seems we are all
> on
> > >> the same page.
> > >>
> > >>
> > >> >> Relating to KIP-448.
> > >> >> What kind of alignment did you think about?
> > >>
> > >> Nothing in particular. Was more or less a random though. Maybe there
> is
> > >> nothing to be alig

Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-06 Thread Jukka Karvanen
Hi,

Now everything what I planned to add including tests, samples and document
changes are in my branch:
https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics


So I can create PR as soon as this KIP is getting green light to proceed.

Jukka

la 4. toukok. 2019 klo 9.05 Jukka Karvanen (jukka.karva...@jukinimi.com)
kirjoitti:

> Hi,
>
> New TestInputTopic and TestOutputTopic included to Developer guide testing
> page as alternative,
> The old way with ConsumerRecordFactory and OutputVerifier is not removed.
>
> You can see the proposal here in my branch:
>
> https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics
>
>
> I can create Work In progress pull request if that make commenting
> proposal easier.
> Still planning to add full coverage unit test and sample WordCountDemoTest to
> streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount,
> if this KIP is accepted.
>
> Jukka
>
>
> ti 30. huhtik. 2019 klo 13.59 Matthias J. Sax (matth...@confluent.io)
> kirjoitti:
>
>> KIP-451 was discarded in favor this this KIP. So it seems we are all on
>> the same page.
>>
>>
>> >> Relating to KIP-448.
>> >> What kind of alignment did you think about?
>>
>> Nothing in particular. Was more or less a random though. Maybe there is
>> nothing to be aligned. Just wanted to bring it up. :)
>>
>>
>> >> Some thoughts after reading also the comments in KAFKA-6460:
>> >> To my understand these KIP-448 mock classes need to be fed somehow into
>> >> TopologyTestDriver.
>> >> I don't know how those KIP-448 mock are planned to be set to
>> >> TopologyTestDriver.
>>
>> KIP-448 is still quite early, and it's a little unclear... Maybe we
>> should just ignore it for now. Sorry for the distraction with my comment
>> about it.
>>
>>
>> Please give me some more time to review this KIP in detail and I will
>> follow up later again.
>>
>>
>> -Matthias
>>
>> On 4/26/19 2:25 PM, Jukka Karvanen wrote:
>> > Yes, my understanding was also that this KIP cover all the requirement
>> of
>> > KIP-451.
>> >
>> > Relating to KIP-448.
>> > What kind of alignment did you think about?
>> >
>> > Some thoughts after reading also the comments in KAFKA-6460:
>> > To my understand these KIP-448 mock classes need to be fed somehow into
>> > TopologyTestDriver.
>> > I don't know how those KIP-448 mock are planned to be set to
>> > TopologyTestDriver.
>> >
>> > On contrast KIP-456 was planned to be on top of unmodified
>> > TopologyTestDriver and now driver is set to TestInputTopic and
>> > TestOutputTopic in constructor.
>> > There are also alternative that these Topic object could be get from
>> > TopologyTestDriver, but it would require the duplicating the
>> constructors
>> > of Topics as methods to
>> > TopologyTestDriver.
>> >
>> > Also related to those Store object when going through the methods in
>> > TopologyTestDriver I noticed accessing the state stores could be be the
>> > third candidate for helper class or a group of classes.
>> > So addition to have TestInputTopic and TestOutputTopic, it could be also
>> > TestKeyValueStore, TestWindowStore, ... to follow the logic in this
>> > KPI-456, but
>> > it does add not any functionality on top of .already existing
>> functionality
>> > *Store classes. So that's why I did not include those.
>> >
>> > Jukka
>> > -
>> >
>> >
>> >
>> >
>> >
>> > Not
>> >
>> > pe 26. huhtik. 2019 klo 12.03 Matthias J. Sax (matth...@confluent.io)
>> > kirjoitti:
>> >
>> >> Btw: there is also KIP-448. I was wondering if it might be required or
>> >> helpful to align the design of both with each other. Thoughts?
>> >>
>> >>
>> >>
>> >> On 4/25/19 11:22 PM, Matthias J. Sax wrote:
>> >>> Thanks for the KIP!
>> >>>
>> >>> I was just comparing this KIP with KIP-451 (even if I did not yet
>> make a
>> >>> sorrow read over this KIP), and I agree that there is a big overlap.
>> It
>> >>> seems that KIP-456 might subsume KIP-451.
>> >>>
>> >>> Let's wait on Patrick's input to see how to proceed.
>> >>>
>> >>>
>> >>> -Matthias
>> >>&

Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-05-04 Thread Jukka Karvanen
Hi,

New TestInputTopic and TestOutputTopic included to Developer guide testing
page as alternative,
The old way with ConsumerRecordFactory and OutputVerifier is not removed.

You can see the proposal here in my branch:
https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics


I can create Work In progress pull request if that make commenting proposal
easier.
Still planning to add full coverage unit test and sample WordCountDemoTest to
streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount,
if this KIP is accepted.

Jukka


ti 30. huhtik. 2019 klo 13.59 Matthias J. Sax (matth...@confluent.io)
kirjoitti:

> KIP-451 was discarded in favor this this KIP. So it seems we are all on
> the same page.
>
>
> >> Relating to KIP-448.
> >> What kind of alignment did you think about?
>
> Nothing in particular. Was more or less a random though. Maybe there is
> nothing to be aligned. Just wanted to bring it up. :)
>
>
> >> Some thoughts after reading also the comments in KAFKA-6460:
> >> To my understand these KIP-448 mock classes need to be fed somehow into
> >> TopologyTestDriver.
> >> I don't know how those KIP-448 mock are planned to be set to
> >> TopologyTestDriver.
>
> KIP-448 is still quite early, and it's a little unclear... Maybe we
> should just ignore it for now. Sorry for the distraction with my comment
> about it.
>
>
> Please give me some more time to review this KIP in detail and I will
> follow up later again.
>
>
> -Matthias
>
> On 4/26/19 2:25 PM, Jukka Karvanen wrote:
> > Yes, my understanding was also that this KIP cover all the requirement of
> > KIP-451.
> >
> > Relating to KIP-448.
> > What kind of alignment did you think about?
> >
> > Some thoughts after reading also the comments in KAFKA-6460:
> > To my understand these KIP-448 mock classes need to be fed somehow into
> > TopologyTestDriver.
> > I don't know how those KIP-448 mock are planned to be set to
> > TopologyTestDriver.
> >
> > On contrast KIP-456 was planned to be on top of unmodified
> > TopologyTestDriver and now driver is set to TestInputTopic and
> > TestOutputTopic in constructor.
> > There are also alternative that these Topic object could be get from
> > TopologyTestDriver, but it would require the duplicating the constructors
> > of Topics as methods to
> > TopologyTestDriver.
> >
> > Also related to those Store object when going through the methods in
> > TopologyTestDriver I noticed accessing the state stores could be be the
> > third candidate for helper class or a group of classes.
> > So addition to have TestInputTopic and TestOutputTopic, it could be also
> > TestKeyValueStore, TestWindowStore, ... to follow the logic in this
> > KPI-456, but
> > it does add not any functionality on top of .already existing
> functionality
> > *Store classes. So that's why I did not include those.
> >
> > Jukka
> > -
> >
> >
> >
> >
> >
> > Not
> >
> > pe 26. huhtik. 2019 klo 12.03 Matthias J. Sax (matth...@confluent.io)
> > kirjoitti:
> >
> >> Btw: there is also KIP-448. I was wondering if it might be required or
> >> helpful to align the design of both with each other. Thoughts?
> >>
> >>
> >>
> >> On 4/25/19 11:22 PM, Matthias J. Sax wrote:
> >>> Thanks for the KIP!
> >>>
> >>> I was just comparing this KIP with KIP-451 (even if I did not yet make
> a
> >>> sorrow read over this KIP), and I agree that there is a big overlap. It
> >>> seems that KIP-456 might subsume KIP-451.
> >>>
> >>> Let's wait on Patrick's input to see how to proceed.
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 4/25/19 12:03 AM, Jukka Karvanen wrote:
> >>>> Hi,
> >>>>
> >>>> If you want to see or test the my current idea of the implementation
> of
> >>>> this KIP, you can check it out in my repo:
> >>>>
> >>
> https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics
> >>>>
> >>>>
> >>>> After my test with KPI-451  I do not see need for add methods for
> >>>> Iterables, but waiting Patrick's clarification of the use case.
> >>>>
> >>>> Jukka
> >>>>
> >>>>
> >>>> ti 23. huhtik. 2019 klo 15.37 Jukka Karvanen (
> >> jukka.karva...@jukinimi.com)
> >&

Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-04-26 Thread Jukka Karvanen
Yes, my understanding was also that this KIP cover all the requirement of
KIP-451.

Relating to KIP-448.
What kind of alignment did you think about?

Some thoughts after reading also the comments in KAFKA-6460:
To my understand these KIP-448 mock classes need to be fed somehow into
TopologyTestDriver.
I don't know how those KIP-448 mock are planned to be set to
TopologyTestDriver.

On contrast KIP-456 was planned to be on top of unmodified
TopologyTestDriver and now driver is set to TestInputTopic and
TestOutputTopic in constructor.
There are also alternative that these Topic object could be get from
TopologyTestDriver, but it would require the duplicating the constructors
of Topics as methods to
TopologyTestDriver.

Also related to those Store object when going through the methods in
TopologyTestDriver I noticed accessing the state stores could be be the
third candidate for helper class or a group of classes.
So addition to have TestInputTopic and TestOutputTopic, it could be also
TestKeyValueStore, TestWindowStore, ... to follow the logic in this
KPI-456, but
it does add not any functionality on top of .already existing functionality
*Store classes. So that's why I did not include those.

Jukka
-





Not

pe 26. huhtik. 2019 klo 12.03 Matthias J. Sax (matth...@confluent.io)
kirjoitti:

> Btw: there is also KIP-448. I was wondering if it might be required or
> helpful to align the design of both with each other. Thoughts?
>
>
>
> On 4/25/19 11:22 PM, Matthias J. Sax wrote:
> > Thanks for the KIP!
> >
> > I was just comparing this KIP with KIP-451 (even if I did not yet make a
> > sorrow read over this KIP), and I agree that there is a big overlap. It
> > seems that KIP-456 might subsume KIP-451.
> >
> > Let's wait on Patrick's input to see how to proceed.
> >
> >
> > -Matthias
> >
> > On 4/25/19 12:03 AM, Jukka Karvanen wrote:
> >> Hi,
> >>
> >> If you want to see or test the my current idea of the implementation of
> >> this KIP, you can check it out in my repo:
> >>
> https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics
> >>
> >>
> >> After my test with KPI-451  I do not see need for add methods for
> >> Iterables, but waiting Patrick's clarification of the use case.
> >>
> >> Jukka
> >>
> >>
> >> ti 23. huhtik. 2019 klo 15.37 Jukka Karvanen (
> jukka.karva...@jukinimi.com)
> >> kirjoitti:
> >>
> >>> Hi All,
> >>>
> >>> I would like to start the discussion on 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
> >>>
> >>>
> >>> There is also related KIP adding methods to TopologyTestDriver:
> >>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> >>>
> >>>
> >>> I added those new Iterable based methods to this TestOutputTopic even
> not
> >>> tested those myself yet.
> >>> So this version contains both my original List and Map based methods
> and
> >>> these new one.
> >>> Based on the discussion some of these can be dropped, if those are
> seen as
> >>> redundant.
> >>>
> >>> Best Regards,
> >>> Jukka
> >>>
> >>>
> >>
> >
>
>


Re: [DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-04-25 Thread Jukka Karvanen
Hi,

If you want to see or test the my current idea of the implementation of
this KIP, you can check it out in my repo:
https://github.com/jukkakarvanen/kafka/compare/trunk...jukkakarvanen:KAFKA-8233_InputOutputTopics


After my test with KPI-451  I do not see need for add methods for
Iterables, but waiting Patrick's clarification of the use case.

Jukka


ti 23. huhtik. 2019 klo 15.37 Jukka Karvanen (jukka.karva...@jukinimi.com)
kirjoitti:

> Hi All,
>
> I would like to start the discussion on 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
>
>
> There is also related KIP adding methods to TopologyTestDriver:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
>
>
> I added those new Iterable based methods to this TestOutputTopic even not
> tested those myself yet.
> So this version contains both my original List and Map based methods and
> these new one.
> Based on the discussion some of these can be dropped, if those are seen as
> redundant.
>
> Best Regards,
> Jukka
>
>


Re: [DISCUSS] KIP-451: Make TopologyTestDriver output iterable

2019-04-25 Thread Jukka Karvanen
Hi,

I played around with Patrick's KAFKA-8200 branch and I tested it with
combined with my draft version of KIP-456.

Some comments:
These two version of iterableOutput methods are working now differently, if
you reuse same fetched Iterable object after piping in new inputs.
Version without serde will see the new input, but version with serdes has
streamed the converted items already to new list and that's why
not seeing the new item. Maybe it is intended to to fetch new Iterable each
time, but the implementation is not mandating it.

See example:
https://github.com/jukkakarvanen/kafka/blob/KAFKA-8200withKIP-456/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverIterableTest.java


I have a lot of tests where I pipe the list of input and check list of
output items, pipe more and check the new list.
Now with this Iterable from the beginning is not very usable if you test
like this in multiple batches. You need to reiterate same again.

In KIP-456 readKeyValuesToList returns List same way ConsumerRecordFactory
and that way this TestInputTopic is accepting List as input.
This collection methods in TestOutputTopic are also consuming the messages.
So you can mix the reading individual rows and collections.
With List it is also easier to get the number of outputs compared to
Iterable.

Please, check out also DISCUSSion of KIP.456. I will post there the link to
the current version of implementation and you can see if it fulfill also
your need.

Jukka


la 20. huhtik. 2019 klo 1.11 Patrik Kleindl (pklei...@gmail.com) kirjoitti:

> Hi Matthias
> Seems I got a bit ahead of myself.
> With option C my aim was a simple alternative which gives back all output
> records that have happened up to this point (and which have not been
> removed by calls to readOutput).
> Based on that the user can decide how to step through or compare the
> records.
>
> If you see it as more consistent if the new methods removed all returned
> records then this can easily be done.
>
> But maybe the pick of Iterable was too narrow.
> It would probably be a good fit to return a List or just a Collection
>
> Picking up John's naming suggestion this would make this:
>
> public Collection> readAllOutput(final
> String topic) {
> final Collection> outputRecords =
> outputRecordsByTopic.get(topic);
> if (outputRecords == null) {
> return Collections.emptyList();
> }
> outputRecordsByTopic.put(topic, new LinkedList<>());
> return outputRecords;
> }
>
> With the semantics the same as readOutput = removing everything.
>
> Can be changed to a List if you think it matters that a user can query
> some index directly.
>
> What do you think?
>
> best regards
>
> Patrik
>
>
>
> On Fri, 19 Apr 2019 at 03:04, Matthias J. Sax 
> wrote:
>
> > I am not sure if (C) is the best option to pick.
> >
> > What is the reasoning to suggest (C) over the other options?
> >
> > It seems that users cannot clear buffered output using option (C). This
> > might it make difficult to write tests.
> >
> > The original Jira tickets suggest:
> >
> > > which returns either an iterator or list over the records that are
> > currently available in the topic
> >
> > This implies that the current buffer would be cleared when getting the
> > iterator.
> >
> > Also, from my understanding, the idea of iterating in general, is to
> > step through a finite collection of objects/elements. Hence, if
> > `hasNext()` returns `false` is will never return `true` later on.
> >
> > As John mentioned, Java also has support for streams, that offer
> > different semantics, that would align with option (C). However, I am not
> > sure if this would be the test API to write tests?
> >
> > Thoughts?
> >
> > In any way: whatever semantics we pick, the KIP should explain them.
> > Atm, this part is missing in the KIP.
> >
> >
> > -Matthias
> >
> > On 4/18/19 12:47 PM, Patrik Kleindl wrote:
> > > Hi John
> > >
> > > Thanks for your feedback
> > > It's C, it does not consume the messages in contrast to the readOutput.
> > > Is it a requirement to do so?
> > > That's why I picked a different name so the difference is more
> > noticeable.
> > > I will add that to the JavaDoc.
> > >
> > > I see your point regarding future changes, that's why I linked KIP-456
> > > where such a method is proposed and would maybe allow to deprecate my
> > > version in favor of a bigger solution.
> > >
> > > Hope that answers your questions
> > >
> > > best regards
> > > Patrik
> > >
> > >
> > > On Thu, 18 Apr 2019 at 19:46, John Roesler  wrote:
> > >
> > >> Hi, Patrik,
> > >>
> > >> Thanks for this proposal!
> > >>
> > >> I have one question, which I didn't see addressed by the KIP.
> Currently,
> > >> when you call `readOutput`, it consumes the result (removes it from
> the
> > >> test driver's output). Does your proposed method:
> > >> A: consume the whole output stream for that topic "atomically" when it
> > >> returns the iterable? (i.e., two calls in a row would guarantee 

[DISCUSS] KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

2019-04-23 Thread Jukka Karvanen
Hi All,

I would like to start the discussion on 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


There is also related KIP adding methods to TopologyTestDriver:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable


I added those new Iterable based methods to this TestOutputTopic even not
tested those myself yet.
So this version contains both my original List and Map based methods and
these new one.
Based on the discussion some of these can be dropped, if those are seen as
redundant.

Best Regards,
Jukka


Request KIP Permissions

2019-04-15 Thread Jukka Karvanen
Hi,

Could you please grant me write access to KIP proposals?
I am planning to make KIP for KAFKA-8233: Helper class to make it simpler
to write test logic with TopologyTestDriver
Wiki ID: jkarvanen

Best regards,
Jukka Karvanen


[jira] [Created] (KAFKA-8233) Helper class to make it simpler to write test logic TopologyTestDriver

2019-04-15 Thread Jukka Karvanen (JIRA)
Jukka Karvanen created KAFKA-8233:
-

 Summary: Helper class to make it simpler to write test logic 
TopologyTestDriver
 Key: KAFKA-8233
 URL: https://issues.apache.org/jira/browse/KAFKA-8233
 Project: Kafka
  Issue Type: Improvement
  Components: streams
Reporter: Jukka Karvanen


When using TopologyTestDriver you need to call ConsumerRecordFactory to create 
ConsumerRecord passed into pipeInput method to write to topic. Also when 
calling readOutput to consume from topic, you need to provide correct 
Deserializers each time.

You easily end up writing helper methods in your test classed, but this can be 
avoided when adding generic input and output topic classes.

This improvement adds TestInputTopic class which wraps TopologyTestDriver  and 
ConsumerRecordFactory methods as one class to be used to write to Input Topics 
and TestOutputTopic class which collects TopologyTestDriver  reading methods 
and provide typesafe read methods.

 

 Example of how Stream test looks after using this classes:

[https://github.com/jukkakarvanen/kafka-streams-examples/blob/InputOutputTopic/src/test/java/io/confluent/examples/streams/WordCountLambdaExampleTest.java]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)