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 <j...@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 <pklei...@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 <pklei...@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 <matth...@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
>>>>>>
>>>>>
>>>>>
>>>
>>
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to