You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Patrik Kleindl <pk...@gmail.com> on 2019/04/11 09:09:37 UTC

[DISCUSS] KIP-451: Make TopologyTestDriver output iterable

Hi everyone,

I would like to start the discussion on this small enhancement of
the TopologyTestDriver.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable

Pull request is available at https://github.com/apache/kafka/pull/6556

Any feedback is welcome

best regards

Patrik

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

Posted by Patrik Kleindl <pk...@gmail.com>.
Hi
It's fine by me, I'll consider doing the PR a good exercise ;-)
I'll discard it for the moment, if there should be any hold-up for KIP-456
to make it into the 2.3 release then we could still revive this one because
it is mostly done already.
best regards and thanks to Jukka for his great work!
Patrik

On Fri, 26 Apr 2019 at 08:19, Matthias J. Sax <ma...@confluent.io> wrote:

> Thanks for the discussion!
>
> After skipping over KIP-456, I was also wondering what the overlap of
> both KIPs are, and if KIP-456 might actually subsume KIP-451.
>
> If we all agree on this, I would recommend to discard KIP-451 in favor
> of KIP-456.
>
> I'll also follow up on KIP-456 DISCUSS thread to get some input about
> the overlap from their point of view.
>
>
> -Matthias
>
> On 4/25/19 2:35 AM, Patrik Kleindl wrote:
> > Hi
> >
> > As discussed, if the preferred option is to consume the records always I
> > will change both methods in KIP-451 accordingly and also switch them to
> > return a List.
> > This would be a bit redundant with Jukkas proposal in KIP-456 so the
> > question is if KIP-451 should be scraped in favor of KIP-456 which has
> more
> > powerful solution but will also need a bit more changes in tests.
> > On the other hand both are useful and wouldn't conflict as far as I can
> see.
> >
> > Any opinions?
> >
> > best regards
> >
> > Patrik
> >
> > On Thu, 25 Apr 2019 at 08:55, Jukka Karvanen <
> jukka.karvanen@jukinimi.com>
> > wrote:
> >
> >> 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 (pkleindl@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<ProducerRecord<byte[], byte[]>> readAllOutput(final
> >>> String topic) {
> >>>     final Collection<ProducerRecord<byte[], byte[]>> 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 <ma...@confluent.io>
> >>> 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 <jo...@confluent.io>
> >> 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 the
> >>>> second
> >>>>>> call is always an empty iterable?)
> >>>>>> B: consume each record when we iterate over it? (i.e., this is like
> >> a
> >>>>>> stream) If this is the case, is the returned object iterable once
> >>>> (uncached
> >>>>>> stream), or could we iterate over it repeatedly (cached stream)?
> >>>>>> C: not consume at all? (i.e., this is a view on the output topic,
> >> but
> >>> we
> >>>>>> need a separate method to consume/clear the output)
> >>>>>> D: something else?
> >>>>>>
> >>>>>> Also, one suggestion: maybe name the method "readAllOutput" or
> >>>> something.
> >>>>>> Specifically naming it "iterable" makes it awkward if we do want to
> >>>> tighten
> >>>>>> the return type (e.g., to List) in the future. This is something we
> >>> may
> >>>>>> actually want to do, if there's an easy way to say, "assert that the
> >>>> output
> >>>>>> equals [...some literal list...]".
> >>>>>>
> >>>>>> Thanks again!
> >>>>>> -John
> >>>>>>
> >>>>>> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com>
> >>>> wrote:
> >>>>>>
> >>>>>>> Hi all
> >>>>>>>
> >>>>>>> Unless someone has objections I will start a VOTE thread tomorrow.
> >>>>>>> The KIP adds two methods to the TopologyTestDriver and has no
> >>> conflicts
> >>>>>> for
> >>>>>>> existing users.
> >>>>>>> PR https://github.com/apache/kafka/pull/6556 is already being
> >>>> reviewed.
> >>>>>>>
> >>>>>>> Side-note:
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
> >>>>>>> will
> >>>>>>> provide a much larger solution for the TopologyTestDriver, but is
> >>> just
> >>>>>>> starting the discussion.
> >>>>>>>
> >>>>>>> best regards
> >>>>>>>
> >>>>>>> Patrik
> >>>>>>>
> >>>>>>> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com>
> >>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Matthias
> >>>>>>>>
> >>>>>>>> Thanks for the questions.
> >>>>>>>>
> >>>>>>>> Regarding the return type:
> >>>>>>>> Iterable offers the option of being used in a foreach loop
> >> directly
> >>>> and
> >>>>>>> it
> >>>>>>>> gives you access to the .iterator method, too.
> >>>>>>>> (ref:
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> >>>>>>>> )
> >>>>>>>>
> >>>>>>>> To return a List object would require an additional conversion
> >> and I
> >>>>>>> don't see the immediate benefit.
> >>>>>>>>
> >>>>>>>> Regarding the ordering:
> >>>>>>>> outputRecordsByTopic gives back a Queue
> >>>>>>>>
> >>>>>>>> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
> >>>>>>> outputRecordsByTopic = new HashMap<>();
> >>>>>>>>
> >>>>>>>> which has a LinkedList behind it
> >>>>>>>>
> >>>>>>>> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
> >>>>>>> LinkedList<>()).add(record);
> >>>>>>>>
> >>>>>>>> So the order is handled by the linked list and should not be
> >>> modified
> >>>>>> by
> >>>>>>>> my changes,
> >>>>>>>> not even the .stream.map etc. (ref:
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> >>>>>>>> )
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Then again, I am open to change it if people have some strong
> >>>>>> preference
> >>>>>>>>
> >>>>>>>> best regards
> >>>>>>>>
> >>>>>>>> Patrik
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <
> >>> matthias@confluent.io>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Thanks for the KIP!
> >>>>>>>>>
> >>>>>>>>> Overall, this makes sense and can simplify testing.
> >>>>>>>>>
> >>>>>>>>> What I am wondering is, why you suggest to return an `Iterable`?
> >>>> Maybe
> >>>>>>>>> returning an `Iterator` would make more sense? Or a List? Note
> >> that
> >>>>>> the
> >>>>>>>>> order of emits matters, thus returning a generic `Collection`
> >> would
> >>>>>> not
> >>>>>>>>> seem to be appropriate.
> >>>>>>>>>
> >>>>>>>>> Can you elaborate on the advantages to use `Iterable` compared to
> >>> the
> >>>>>>>>> other options?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> >>>>>>>>>> Hi everyone,
> >>>>>>>>>>
> >>>>>>>>>> I would like to start the discussion on this small enhancement
> >> of
> >>>>>>>>>> the TopologyTestDriver.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> >>>>>>>>>>
> >>>>>>>>>> Pull request is available at
> >>>>>>> https://github.com/apache/kafka/pull/6556
> >>>>>>>>>>
> >>>>>>>>>> Any feedback is welcome
> >>>>>>>>>>
> >>>>>>>>>> best regards
> >>>>>>>>>>
> >>>>>>>>>> Patrik
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>
> >
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for the discussion!

After skipping over KIP-456, I was also wondering what the overlap of
both KIPs are, and if KIP-456 might actually subsume KIP-451.

If we all agree on this, I would recommend to discard KIP-451 in favor
of KIP-456.

I'll also follow up on KIP-456 DISCUSS thread to get some input about
the overlap from their point of view.


-Matthias

On 4/25/19 2:35 AM, Patrik Kleindl wrote:
> Hi
> 
> As discussed, if the preferred option is to consume the records always I
> will change both methods in KIP-451 accordingly and also switch them to
> return a List.
> This would be a bit redundant with Jukkas proposal in KIP-456 so the
> question is if KIP-451 should be scraped in favor of KIP-456 which has more
> powerful solution but will also need a bit more changes in tests.
> On the other hand both are useful and wouldn't conflict as far as I can see.
> 
> Any opinions?
> 
> best regards
> 
> Patrik
> 
> On Thu, 25 Apr 2019 at 08:55, Jukka Karvanen <ju...@jukinimi.com>
> wrote:
> 
>> 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 (pkleindl@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<ProducerRecord<byte[], byte[]>> readAllOutput(final
>>> String topic) {
>>>     final Collection<ProducerRecord<byte[], byte[]>> 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 <ma...@confluent.io>
>>> 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 <jo...@confluent.io>
>> 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 the
>>>> second
>>>>>> call is always an empty iterable?)
>>>>>> B: consume each record when we iterate over it? (i.e., this is like
>> a
>>>>>> stream) If this is the case, is the returned object iterable once
>>>> (uncached
>>>>>> stream), or could we iterate over it repeatedly (cached stream)?
>>>>>> C: not consume at all? (i.e., this is a view on the output topic,
>> but
>>> we
>>>>>> need a separate method to consume/clear the output)
>>>>>> D: something else?
>>>>>>
>>>>>> Also, one suggestion: maybe name the method "readAllOutput" or
>>>> something.
>>>>>> Specifically naming it "iterable" makes it awkward if we do want to
>>>> tighten
>>>>>> the return type (e.g., to List) in the future. This is something we
>>> may
>>>>>> actually want to do, if there's an easy way to say, "assert that the
>>>> output
>>>>>> equals [...some literal list...]".
>>>>>>
>>>>>> Thanks again!
>>>>>> -John
>>>>>>
>>>>>> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com>
>>>> wrote:
>>>>>>
>>>>>>> Hi all
>>>>>>>
>>>>>>> Unless someone has objections I will start a VOTE thread tomorrow.
>>>>>>> The KIP adds two methods to the TopologyTestDriver and has no
>>> conflicts
>>>>>> for
>>>>>>> existing users.
>>>>>>> PR https://github.com/apache/kafka/pull/6556 is already being
>>>> reviewed.
>>>>>>>
>>>>>>> Side-note:
>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
>>>>>>> will
>>>>>>> provide a much larger solution for the TopologyTestDriver, but is
>>> just
>>>>>>> starting the discussion.
>>>>>>>
>>>>>>> best regards
>>>>>>>
>>>>>>> Patrik
>>>>>>>
>>>>>>> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com>
>>>> wrote:
>>>>>>>
>>>>>>>> Hi Matthias
>>>>>>>>
>>>>>>>> Thanks for the questions.
>>>>>>>>
>>>>>>>> Regarding the return type:
>>>>>>>> Iterable offers the option of being used in a foreach loop
>> directly
>>>> and
>>>>>>> it
>>>>>>>> gives you access to the .iterator method, too.
>>>>>>>> (ref:
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
>>>>>>>> )
>>>>>>>>
>>>>>>>> To return a List object would require an additional conversion
>> and I
>>>>>>> don't see the immediate benefit.
>>>>>>>>
>>>>>>>> Regarding the ordering:
>>>>>>>> outputRecordsByTopic gives back a Queue
>>>>>>>>
>>>>>>>> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
>>>>>>> outputRecordsByTopic = new HashMap<>();
>>>>>>>>
>>>>>>>> which has a LinkedList behind it
>>>>>>>>
>>>>>>>> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
>>>>>>> LinkedList<>()).add(record);
>>>>>>>>
>>>>>>>> So the order is handled by the linked list and should not be
>>> modified
>>>>>> by
>>>>>>>> my changes,
>>>>>>>> not even the .stream.map etc. (ref:
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
>>>>>>>> )
>>>>>>>>
>>>>>>>>
>>>>>>>> Then again, I am open to change it if people have some strong
>>>>>> preference
>>>>>>>>
>>>>>>>> best regards
>>>>>>>>
>>>>>>>> Patrik
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <
>>> matthias@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Thanks for the KIP!
>>>>>>>>>
>>>>>>>>> Overall, this makes sense and can simplify testing.
>>>>>>>>>
>>>>>>>>> What I am wondering is, why you suggest to return an `Iterable`?
>>>> Maybe
>>>>>>>>> returning an `Iterator` would make more sense? Or a List? Note
>> that
>>>>>> the
>>>>>>>>> order of emits matters, thus returning a generic `Collection`
>> would
>>>>>> not
>>>>>>>>> seem to be appropriate.
>>>>>>>>>
>>>>>>>>> Can you elaborate on the advantages to use `Iterable` compared to
>>> the
>>>>>>>>> other options?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
>>>>>>>>>> Hi everyone,
>>>>>>>>>>
>>>>>>>>>> I would like to start the discussion on this small enhancement
>> of
>>>>>>>>>> the TopologyTestDriver.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
>>>>>>>>>>
>>>>>>>>>> Pull request is available at
>>>>>>> https://github.com/apache/kafka/pull/6556
>>>>>>>>>>
>>>>>>>>>> Any feedback is welcome
>>>>>>>>>>
>>>>>>>>>> best regards
>>>>>>>>>>
>>>>>>>>>> Patrik
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>
>>
> 


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

Posted by Patrik Kleindl <pk...@gmail.com>.
Hi

As discussed, if the preferred option is to consume the records always I
will change both methods in KIP-451 accordingly and also switch them to
return a List.
This would be a bit redundant with Jukkas proposal in KIP-456 so the
question is if KIP-451 should be scraped in favor of KIP-456 which has more
powerful solution but will also need a bit more changes in tests.
On the other hand both are useful and wouldn't conflict as far as I can see.

Any opinions?

best regards

Patrik

On Thu, 25 Apr 2019 at 08:55, Jukka Karvanen <ju...@jukinimi.com>
wrote:

> 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 (pkleindl@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<ProducerRecord<byte[], byte[]>> readAllOutput(final
> > String topic) {
> >     final Collection<ProducerRecord<byte[], byte[]>> 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 <ma...@confluent.io>
> > 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 <jo...@confluent.io>
> 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 the
> > > second
> > > >> call is always an empty iterable?)
> > > >> B: consume each record when we iterate over it? (i.e., this is like
> a
> > > >> stream) If this is the case, is the returned object iterable once
> > > (uncached
> > > >> stream), or could we iterate over it repeatedly (cached stream)?
> > > >> C: not consume at all? (i.e., this is a view on the output topic,
> but
> > we
> > > >> need a separate method to consume/clear the output)
> > > >> D: something else?
> > > >>
> > > >> Also, one suggestion: maybe name the method "readAllOutput" or
> > > something.
> > > >> Specifically naming it "iterable" makes it awkward if we do want to
> > > tighten
> > > >> the return type (e.g., to List) in the future. This is something we
> > may
> > > >> actually want to do, if there's an easy way to say, "assert that the
> > > output
> > > >> equals [...some literal list...]".
> > > >>
> > > >> Thanks again!
> > > >> -John
> > > >>
> > > >> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com>
> > > wrote:
> > > >>
> > > >>> Hi all
> > > >>>
> > > >>> Unless someone has objections I will start a VOTE thread tomorrow.
> > > >>> The KIP adds two methods to the TopologyTestDriver and has no
> > conflicts
> > > >> for
> > > >>> existing users.
> > > >>> PR https://github.com/apache/kafka/pull/6556 is already being
> > > reviewed.
> > > >>>
> > > >>> Side-note:
> > > >>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
> > > >>> will
> > > >>> provide a much larger solution for the TopologyTestDriver, but is
> > just
> > > >>> starting the discussion.
> > > >>>
> > > >>> best regards
> > > >>>
> > > >>> Patrik
> > > >>>
> > > >>> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com>
> > > wrote:
> > > >>>
> > > >>>> Hi Matthias
> > > >>>>
> > > >>>> Thanks for the questions.
> > > >>>>
> > > >>>> Regarding the return type:
> > > >>>> Iterable offers the option of being used in a foreach loop
> directly
> > > and
> > > >>> it
> > > >>>> gives you access to the .iterator method, too.
> > > >>>> (ref:
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> > > >>>> )
> > > >>>>
> > > >>>> To return a List object would require an additional conversion
> and I
> > > >>> don't see the immediate benefit.
> > > >>>>
> > > >>>> Regarding the ordering:
> > > >>>> outputRecordsByTopic gives back a Queue
> > > >>>>
> > > >>>> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
> > > >>> outputRecordsByTopic = new HashMap<>();
> > > >>>>
> > > >>>> which has a LinkedList behind it
> > > >>>>
> > > >>>> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
> > > >>> LinkedList<>()).add(record);
> > > >>>>
> > > >>>> So the order is handled by the linked list and should not be
> > modified
> > > >> by
> > > >>>> my changes,
> > > >>>> not even the .stream.map etc. (ref:
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> > > >>>> )
> > > >>>>
> > > >>>>
> > > >>>> Then again, I am open to change it if people have some strong
> > > >> preference
> > > >>>>
> > > >>>> best regards
> > > >>>>
> > > >>>> Patrik
> > > >>>>
> > > >>>>
> > > >>>> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <
> > matthias@confluent.io>
> > > >>>> wrote:
> > > >>>>
> > > >>>>> Thanks for the KIP!
> > > >>>>>
> > > >>>>> Overall, this makes sense and can simplify testing.
> > > >>>>>
> > > >>>>> What I am wondering is, why you suggest to return an `Iterable`?
> > > Maybe
> > > >>>>> returning an `Iterator` would make more sense? Or a List? Note
> that
> > > >> the
> > > >>>>> order of emits matters, thus returning a generic `Collection`
> would
> > > >> not
> > > >>>>> seem to be appropriate.
> > > >>>>>
> > > >>>>> Can you elaborate on the advantages to use `Iterable` compared to
> > the
> > > >>>>> other options?
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> -Matthias
> > > >>>>>
> > > >>>>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> > > >>>>>> Hi everyone,
> > > >>>>>>
> > > >>>>>> I would like to start the discussion on this small enhancement
> of
> > > >>>>>> the TopologyTestDriver.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> > > >>>>>>
> > > >>>>>> Pull request is available at
> > > >>> https://github.com/apache/kafka/pull/6556
> > > >>>>>>
> > > >>>>>> Any feedback is welcome
> > > >>>>>>
> > > >>>>>> best regards
> > > >>>>>>
> > > >>>>>> Patrik
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>
> > > >>
> > > >
> > >
> > >
> >
>

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

Posted by Jukka Karvanen <ju...@jukinimi.com>.
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 (pkleindl@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<ProducerRecord<byte[], byte[]>> readAllOutput(final
> String topic) {
>     final Collection<ProducerRecord<byte[], byte[]>> 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 <ma...@confluent.io>
> 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 <jo...@confluent.io> 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 the
> > second
> > >> call is always an empty iterable?)
> > >> B: consume each record when we iterate over it? (i.e., this is like a
> > >> stream) If this is the case, is the returned object iterable once
> > (uncached
> > >> stream), or could we iterate over it repeatedly (cached stream)?
> > >> C: not consume at all? (i.e., this is a view on the output topic, but
> we
> > >> need a separate method to consume/clear the output)
> > >> D: something else?
> > >>
> > >> Also, one suggestion: maybe name the method "readAllOutput" or
> > something.
> > >> Specifically naming it "iterable" makes it awkward if we do want to
> > tighten
> > >> the return type (e.g., to List) in the future. This is something we
> may
> > >> actually want to do, if there's an easy way to say, "assert that the
> > output
> > >> equals [...some literal list...]".
> > >>
> > >> Thanks again!
> > >> -John
> > >>
> > >> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com>
> > wrote:
> > >>
> > >>> Hi all
> > >>>
> > >>> Unless someone has objections I will start a VOTE thread tomorrow.
> > >>> The KIP adds two methods to the TopologyTestDriver and has no
> conflicts
> > >> for
> > >>> existing users.
> > >>> PR https://github.com/apache/kafka/pull/6556 is already being
> > reviewed.
> > >>>
> > >>> Side-note:
> > >>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
> > >>> will
> > >>> provide a much larger solution for the TopologyTestDriver, but is
> just
> > >>> starting the discussion.
> > >>>
> > >>> best regards
> > >>>
> > >>> Patrik
> > >>>
> > >>> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com>
> > wrote:
> > >>>
> > >>>> Hi Matthias
> > >>>>
> > >>>> Thanks for the questions.
> > >>>>
> > >>>> Regarding the return type:
> > >>>> Iterable offers the option of being used in a foreach loop directly
> > and
> > >>> it
> > >>>> gives you access to the .iterator method, too.
> > >>>> (ref:
> > >>>>
> > >>>
> > >>
> >
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> > >>>> )
> > >>>>
> > >>>> To return a List object would require an additional conversion and I
> > >>> don't see the immediate benefit.
> > >>>>
> > >>>> Regarding the ordering:
> > >>>> outputRecordsByTopic gives back a Queue
> > >>>>
> > >>>> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
> > >>> outputRecordsByTopic = new HashMap<>();
> > >>>>
> > >>>> which has a LinkedList behind it
> > >>>>
> > >>>> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
> > >>> LinkedList<>()).add(record);
> > >>>>
> > >>>> So the order is handled by the linked list and should not be
> modified
> > >> by
> > >>>> my changes,
> > >>>> not even the .stream.map etc. (ref:
> > >>>>
> > >>>
> > >>
> >
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> > >>>> )
> > >>>>
> > >>>>
> > >>>> Then again, I am open to change it if people have some strong
> > >> preference
> > >>>>
> > >>>> best regards
> > >>>>
> > >>>> Patrik
> > >>>>
> > >>>>
> > >>>> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <
> matthias@confluent.io>
> > >>>> wrote:
> > >>>>
> > >>>>> Thanks for the KIP!
> > >>>>>
> > >>>>> Overall, this makes sense and can simplify testing.
> > >>>>>
> > >>>>> What I am wondering is, why you suggest to return an `Iterable`?
> > Maybe
> > >>>>> returning an `Iterator` would make more sense? Or a List? Note that
> > >> the
> > >>>>> order of emits matters, thus returning a generic `Collection` would
> > >> not
> > >>>>> seem to be appropriate.
> > >>>>>
> > >>>>> Can you elaborate on the advantages to use `Iterable` compared to
> the
> > >>>>> other options?
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> -Matthias
> > >>>>>
> > >>>>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> > >>>>>> Hi everyone,
> > >>>>>>
> > >>>>>> I would like to start the discussion on this small enhancement of
> > >>>>>> the TopologyTestDriver.
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> > >>>>>>
> > >>>>>> Pull request is available at
> > >>> https://github.com/apache/kafka/pull/6556
> > >>>>>>
> > >>>>>> Any feedback is welcome
> > >>>>>>
> > >>>>>> best regards
> > >>>>>>
> > >>>>>> Patrik
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>
> > >>
> > >
> >
> >
>

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

Posted by Patrik Kleindl <pk...@gmail.com>.
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<ProducerRecord<byte[], byte[]>> readAllOutput(final
String topic) {
    final Collection<ProducerRecord<byte[], byte[]>> 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 <ma...@confluent.io> 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 <jo...@confluent.io> 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 the
> second
> >> call is always an empty iterable?)
> >> B: consume each record when we iterate over it? (i.e., this is like a
> >> stream) If this is the case, is the returned object iterable once
> (uncached
> >> stream), or could we iterate over it repeatedly (cached stream)?
> >> C: not consume at all? (i.e., this is a view on the output topic, but we
> >> need a separate method to consume/clear the output)
> >> D: something else?
> >>
> >> Also, one suggestion: maybe name the method "readAllOutput" or
> something.
> >> Specifically naming it "iterable" makes it awkward if we do want to
> tighten
> >> the return type (e.g., to List) in the future. This is something we may
> >> actually want to do, if there's an easy way to say, "assert that the
> output
> >> equals [...some literal list...]".
> >>
> >> Thanks again!
> >> -John
> >>
> >> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com>
> wrote:
> >>
> >>> Hi all
> >>>
> >>> Unless someone has objections I will start a VOTE thread tomorrow.
> >>> The KIP adds two methods to the TopologyTestDriver and has no conflicts
> >> for
> >>> existing users.
> >>> PR https://github.com/apache/kafka/pull/6556 is already being
> reviewed.
> >>>
> >>> Side-note:
> >>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
> >>> will
> >>> provide a much larger solution for the TopologyTestDriver, but is just
> >>> starting the discussion.
> >>>
> >>> best regards
> >>>
> >>> Patrik
> >>>
> >>> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com>
> wrote:
> >>>
> >>>> Hi Matthias
> >>>>
> >>>> Thanks for the questions.
> >>>>
> >>>> Regarding the return type:
> >>>> Iterable offers the option of being used in a foreach loop directly
> and
> >>> it
> >>>> gives you access to the .iterator method, too.
> >>>> (ref:
> >>>>
> >>>
> >>
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> >>>> )
> >>>>
> >>>> To return a List object would require an additional conversion and I
> >>> don't see the immediate benefit.
> >>>>
> >>>> Regarding the ordering:
> >>>> outputRecordsByTopic gives back a Queue
> >>>>
> >>>> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
> >>> outputRecordsByTopic = new HashMap<>();
> >>>>
> >>>> which has a LinkedList behind it
> >>>>
> >>>> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
> >>> LinkedList<>()).add(record);
> >>>>
> >>>> So the order is handled by the linked list and should not be modified
> >> by
> >>>> my changes,
> >>>> not even the .stream.map etc. (ref:
> >>>>
> >>>
> >>
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> >>>> )
> >>>>
> >>>>
> >>>> Then again, I am open to change it if people have some strong
> >> preference
> >>>>
> >>>> best regards
> >>>>
> >>>> Patrik
> >>>>
> >>>>
> >>>> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <ma...@confluent.io>
> >>>> wrote:
> >>>>
> >>>>> Thanks for the KIP!
> >>>>>
> >>>>> Overall, this makes sense and can simplify testing.
> >>>>>
> >>>>> What I am wondering is, why you suggest to return an `Iterable`?
> Maybe
> >>>>> returning an `Iterator` would make more sense? Or a List? Note that
> >> the
> >>>>> order of emits matters, thus returning a generic `Collection` would
> >> not
> >>>>> seem to be appropriate.
> >>>>>
> >>>>> Can you elaborate on the advantages to use `Iterable` compared to the
> >>>>> other options?
> >>>>>
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> >>>>>> Hi everyone,
> >>>>>>
> >>>>>> I would like to start the discussion on this small enhancement of
> >>>>>> the TopologyTestDriver.
> >>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> >>>>>>
> >>>>>> Pull request is available at
> >>> https://github.com/apache/kafka/pull/6556
> >>>>>>
> >>>>>> Any feedback is welcome
> >>>>>>
> >>>>>> best regards
> >>>>>>
> >>>>>> Patrik
> >>>>>>
> >>>>>
> >>>>>
> >>>
> >>
> >
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 <jo...@confluent.io> 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 the second
>> call is always an empty iterable?)
>> B: consume each record when we iterate over it? (i.e., this is like a
>> stream) If this is the case, is the returned object iterable once (uncached
>> stream), or could we iterate over it repeatedly (cached stream)?
>> C: not consume at all? (i.e., this is a view on the output topic, but we
>> need a separate method to consume/clear the output)
>> D: something else?
>>
>> Also, one suggestion: maybe name the method "readAllOutput" or something.
>> Specifically naming it "iterable" makes it awkward if we do want to tighten
>> the return type (e.g., to List) in the future. This is something we may
>> actually want to do, if there's an easy way to say, "assert that the output
>> equals [...some literal list...]".
>>
>> Thanks again!
>> -John
>>
>> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com> wrote:
>>
>>> Hi all
>>>
>>> Unless someone has objections I will start a VOTE thread tomorrow.
>>> The KIP adds two methods to the TopologyTestDriver and has no conflicts
>> for
>>> existing users.
>>> PR https://github.com/apache/kafka/pull/6556 is already being reviewed.
>>>
>>> Side-note:
>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
>>> will
>>> provide a much larger solution for the TopologyTestDriver, but is just
>>> starting the discussion.
>>>
>>> best regards
>>>
>>> Patrik
>>>
>>> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com> wrote:
>>>
>>>> Hi Matthias
>>>>
>>>> Thanks for the questions.
>>>>
>>>> Regarding the return type:
>>>> Iterable offers the option of being used in a foreach loop directly and
>>> it
>>>> gives you access to the .iterator method, too.
>>>> (ref:
>>>>
>>>
>> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
>>>> )
>>>>
>>>> To return a List object would require an additional conversion and I
>>> don't see the immediate benefit.
>>>>
>>>> Regarding the ordering:
>>>> outputRecordsByTopic gives back a Queue
>>>>
>>>> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
>>> outputRecordsByTopic = new HashMap<>();
>>>>
>>>> which has a LinkedList behind it
>>>>
>>>> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
>>> LinkedList<>()).add(record);
>>>>
>>>> So the order is handled by the linked list and should not be modified
>> by
>>>> my changes,
>>>> not even the .stream.map etc. (ref:
>>>>
>>>
>> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
>>>> )
>>>>
>>>>
>>>> Then again, I am open to change it if people have some strong
>> preference
>>>>
>>>> best regards
>>>>
>>>> Patrik
>>>>
>>>>
>>>> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>
>>>>> Thanks for the KIP!
>>>>>
>>>>> Overall, this makes sense and can simplify testing.
>>>>>
>>>>> What I am wondering is, why you suggest to return an `Iterable`? Maybe
>>>>> returning an `Iterator` would make more sense? Or a List? Note that
>> the
>>>>> order of emits matters, thus returning a generic `Collection` would
>> not
>>>>> seem to be appropriate.
>>>>>
>>>>> Can you elaborate on the advantages to use `Iterable` compared to the
>>>>> other options?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
>>>>>> Hi everyone,
>>>>>>
>>>>>> I would like to start the discussion on this small enhancement of
>>>>>> the TopologyTestDriver.
>>>>>>
>>>>>>
>>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
>>>>>>
>>>>>> Pull request is available at
>>> https://github.com/apache/kafka/pull/6556
>>>>>>
>>>>>> Any feedback is welcome
>>>>>>
>>>>>> best regards
>>>>>>
>>>>>> Patrik
>>>>>>
>>>>>
>>>>>
>>>
>>
> 


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

Posted by Patrik Kleindl <pk...@gmail.com>.
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 <jo...@confluent.io> 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 the second
> call is always an empty iterable?)
> B: consume each record when we iterate over it? (i.e., this is like a
> stream) If this is the case, is the returned object iterable once (uncached
> stream), or could we iterate over it repeatedly (cached stream)?
> C: not consume at all? (i.e., this is a view on the output topic, but we
> need a separate method to consume/clear the output)
> D: something else?
>
> Also, one suggestion: maybe name the method "readAllOutput" or something.
> Specifically naming it "iterable" makes it awkward if we do want to tighten
> the return type (e.g., to List) in the future. This is something we may
> actually want to do, if there's an easy way to say, "assert that the output
> equals [...some literal list...]".
>
> Thanks again!
> -John
>
> On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com> wrote:
>
> > Hi all
> >
> > Unless someone has objections I will start a VOTE thread tomorrow.
> > The KIP adds two methods to the TopologyTestDriver and has no conflicts
> for
> > existing users.
> > PR https://github.com/apache/kafka/pull/6556 is already being reviewed.
> >
> > Side-note:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
> > will
> > provide a much larger solution for the TopologyTestDriver, but is just
> > starting the discussion.
> >
> > best regards
> >
> > Patrik
> >
> > On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com> wrote:
> >
> > > Hi Matthias
> > >
> > > Thanks for the questions.
> > >
> > > Regarding the return type:
> > > Iterable offers the option of being used in a foreach loop directly and
> > it
> > > gives you access to the .iterator method, too.
> > > (ref:
> > >
> >
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> > > )
> > >
> > > To return a List object would require an additional conversion and I
> > don't see the immediate benefit.
> > >
> > > Regarding the ordering:
> > > outputRecordsByTopic gives back a Queue
> > >
> > > private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
> > outputRecordsByTopic = new HashMap<>();
> > >
> > > which has a LinkedList behind it
> > >
> > > outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
> > LinkedList<>()).add(record);
> > >
> > > So the order is handled by the linked list and should not be modified
> by
> > > my changes,
> > > not even the .stream.map etc. (ref:
> > >
> >
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> > > )
> > >
> > >
> > > Then again, I am open to change it if people have some strong
> preference
> > >
> > > best regards
> > >
> > > Patrik
> > >
> > >
> > > On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <ma...@confluent.io>
> > > wrote:
> > >
> > >> Thanks for the KIP!
> > >>
> > >> Overall, this makes sense and can simplify testing.
> > >>
> > >> What I am wondering is, why you suggest to return an `Iterable`? Maybe
> > >> returning an `Iterator` would make more sense? Or a List? Note that
> the
> > >> order of emits matters, thus returning a generic `Collection` would
> not
> > >> seem to be appropriate.
> > >>
> > >> Can you elaborate on the advantages to use `Iterable` compared to the
> > >> other options?
> > >>
> > >>
> > >>
> > >> -Matthias
> > >>
> > >> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> > >> > Hi everyone,
> > >> >
> > >> > I would like to start the discussion on this small enhancement of
> > >> > the TopologyTestDriver.
> > >> >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> > >> >
> > >> > Pull request is available at
> > https://github.com/apache/kafka/pull/6556
> > >> >
> > >> > Any feedback is welcome
> > >> >
> > >> > best regards
> > >> >
> > >> > Patrik
> > >> >
> > >>
> > >>
> >
>

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

Posted by John Roesler <jo...@confluent.io>.
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 the second
call is always an empty iterable?)
B: consume each record when we iterate over it? (i.e., this is like a
stream) If this is the case, is the returned object iterable once (uncached
stream), or could we iterate over it repeatedly (cached stream)?
C: not consume at all? (i.e., this is a view on the output topic, but we
need a separate method to consume/clear the output)
D: something else?

Also, one suggestion: maybe name the method "readAllOutput" or something.
Specifically naming it "iterable" makes it awkward if we do want to tighten
the return type (e.g., to List) in the future. This is something we may
actually want to do, if there's an easy way to say, "assert that the output
equals [...some literal list...]".

Thanks again!
-John

On Wed, Apr 17, 2019 at 4:01 PM Patrik Kleindl <pk...@gmail.com> wrote:

> Hi all
>
> Unless someone has objections I will start a VOTE thread tomorrow.
> The KIP adds two methods to the TopologyTestDriver and has no conflicts for
> existing users.
> PR https://github.com/apache/kafka/pull/6556 is already being reviewed.
>
> Side-note:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
> will
> provide a much larger solution for the TopologyTestDriver, but is just
> starting the discussion.
>
> best regards
>
> Patrik
>
> On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com> wrote:
>
> > Hi Matthias
> >
> > Thanks for the questions.
> >
> > Regarding the return type:
> > Iterable offers the option of being used in a foreach loop directly and
> it
> > gives you access to the .iterator method, too.
> > (ref:
> >
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> > )
> >
> > To return a List object would require an additional conversion and I
> don't see the immediate benefit.
> >
> > Regarding the ordering:
> > outputRecordsByTopic gives back a Queue
> >
> > private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
> outputRecordsByTopic = new HashMap<>();
> >
> > which has a LinkedList behind it
> >
> > outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
> LinkedList<>()).add(record);
> >
> > So the order is handled by the linked list and should not be modified by
> > my changes,
> > not even the .stream.map etc. (ref:
> >
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> > )
> >
> >
> > Then again, I am open to change it if people have some strong preference
> >
> > best regards
> >
> > Patrik
> >
> >
> > On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> Thanks for the KIP!
> >>
> >> Overall, this makes sense and can simplify testing.
> >>
> >> What I am wondering is, why you suggest to return an `Iterable`? Maybe
> >> returning an `Iterator` would make more sense? Or a List? Note that the
> >> order of emits matters, thus returning a generic `Collection` would not
> >> seem to be appropriate.
> >>
> >> Can you elaborate on the advantages to use `Iterable` compared to the
> >> other options?
> >>
> >>
> >>
> >> -Matthias
> >>
> >> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> >> > Hi everyone,
> >> >
> >> > I would like to start the discussion on this small enhancement of
> >> > the TopologyTestDriver.
> >> >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> >> >
> >> > Pull request is available at
> https://github.com/apache/kafka/pull/6556
> >> >
> >> > Any feedback is welcome
> >> >
> >> > best regards
> >> >
> >> > Patrik
> >> >
> >>
> >>
>

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

Posted by Patrik Kleindl <pk...@gmail.com>.
Hi all

Unless someone has objections I will start a VOTE thread tomorrow.
The KIP adds two methods to the TopologyTestDriver and has no conflicts for
existing users.
PR https://github.com/apache/kafka/pull/6556 is already being reviewed.

Side-note:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-456%3A+Helper+classes+to+make+it+simpler+to+write+test+logic+with+TopologyTestDriver
will
provide a much larger solution for the TopologyTestDriver, but is just
starting the discussion.

best regards

Patrik

On Thu, 11 Apr 2019 at 22:14, Patrik Kleindl <pk...@gmail.com> wrote:

> Hi Matthias
>
> Thanks for the questions.
>
> Regarding the return type:
> Iterable offers the option of being used in a foreach loop directly and it
> gives you access to the .iterator method, too.
> (ref:
> https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
> )
>
> To return a List object would require an additional conversion and I don't see the immediate benefit.
>
> Regarding the ordering:
> outputRecordsByTopic gives back a Queue
>
> private final Map<String, Queue<ProducerRecord<byte[], byte[]>>> outputRecordsByTopic = new HashMap<>();
>
> which has a LinkedList behind it
>
> outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new LinkedList<>()).add(record);
>
> So the order is handled by the linked list and should not be modified by
> my changes,
> not even the .stream.map etc. (ref:
> https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
> )
>
>
> Then again, I am open to change it if people have some strong preference
>
> best regards
>
> Patrik
>
>
> On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> Thanks for the KIP!
>>
>> Overall, this makes sense and can simplify testing.
>>
>> What I am wondering is, why you suggest to return an `Iterable`? Maybe
>> returning an `Iterator` would make more sense? Or a List? Note that the
>> order of emits matters, thus returning a generic `Collection` would not
>> seem to be appropriate.
>>
>> Can you elaborate on the advantages to use `Iterable` compared to the
>> other options?
>>
>>
>>
>> -Matthias
>>
>> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
>> > Hi everyone,
>> >
>> > I would like to start the discussion on this small enhancement of
>> > the TopologyTestDriver.
>> >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
>> >
>> > Pull request is available at https://github.com/apache/kafka/pull/6556
>> >
>> > Any feedback is welcome
>> >
>> > best regards
>> >
>> > Patrik
>> >
>>
>>

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

Posted by Patrik Kleindl <pk...@gmail.com>.
Hi Matthias

Thanks for the questions.

Regarding the return type:
Iterable offers the option of being used in a foreach loop directly and it
gives you access to the .iterator method, too.
(ref:
https://www.techiedelight.com/differences-between-iterator-and-iterable-in-java/
)

To return a List object would require an additional conversion and I
don't see the immediate benefit.

Regarding the ordering:
outputRecordsByTopic gives back a Queue

private final Map<String, Queue<ProducerRecord<byte[], byte[]>>>
outputRecordsByTopic = new HashMap<>();

which has a LinkedList behind it

outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new
LinkedList<>()).add(record);

So the order is handled by the linked list and should not be modified by my
changes,
not even the .stream.map etc. (ref:
https://stackoverflow.com/questions/30258566/java-stream-map-and-collect-order-of-resulting-container
)


Then again, I am open to change it if people have some strong preference

best regards

Patrik


On Thu, 11 Apr 2019 at 17:45, Matthias J. Sax <ma...@confluent.io> wrote:

> Thanks for the KIP!
>
> Overall, this makes sense and can simplify testing.
>
> What I am wondering is, why you suggest to return an `Iterable`? Maybe
> returning an `Iterator` would make more sense? Or a List? Note that the
> order of emits matters, thus returning a generic `Collection` would not
> seem to be appropriate.
>
> Can you elaborate on the advantages to use `Iterable` compared to the
> other options?
>
>
>
> -Matthias
>
> On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> > Hi everyone,
> >
> > I would like to start the discussion on this small enhancement of
> > the TopologyTestDriver.
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> >
> > Pull request is available at https://github.com/apache/kafka/pull/6556
> >
> > Any feedback is welcome
> >
> > best regards
> >
> > Patrik
> >
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for the KIP!

Overall, this makes sense and can simplify testing.

What I am wondering is, why you suggest to return an `Iterable`? Maybe
returning an `Iterator` would make more sense? Or a List? Note that the
order of emits matters, thus returning a generic `Collection` would not
seem to be appropriate.

Can you elaborate on the advantages to use `Iterable` compared to the
other options?



-Matthias

On 4/11/19 2:09 AM, Patrik Kleindl wrote:
> Hi everyone,
> 
> I would like to start the discussion on this small enhancement of
> the TopologyTestDriver.
> 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-451%3A+Make+TopologyTestDriver+output+iterable
> 
> Pull request is available at https://github.com/apache/kafka/pull/6556
> 
> Any feedback is welcome
> 
> best regards
> 
> Patrik
>