Thanks for the KIP John.

Couple of minor questions:

- What about putting the mock into sub-package `processor` so it's in
the same package name as the interface it implements?

- What is the purpose of the constructor talking the `File stateDir`
argument? The state directory should be encoded in the `Properties
config' parameter already.

- We have KIP-251 that place (not voted yet though) that plans to
deprecate `forward(K key, V value, int childIndex)` and `forward(K key,
V value, String childName)`  -- should we also throw
UnsupportedOperationException similar to `schedule(long)` if KIP-251 is
accepted?


-Matthias

On 3/8/18 3:16 PM, John Roesler wrote:
> Thanks for the review, Guozhang,
> 
> In response:
> 1. I missed that! I'll look into it and update the KIP.
> 
> 2. I was planning to use the real implementation, since folks might
> register some metrics in the processors and want to verify the values that
> get recorded. If the concern is about initializing all the stuff that's in
> the Metrics object, I can instantiate it lazily or even make it optional by
> taking a nullable constructor parameter.
> 
> 3. Agreed. I think that's the real sharp edge here. I actually think it
> would be neat to auto-trigger those scheduled punctuators, but it seems
> like that moves this component out of "mock" territory and into "driver"
> territory. Since we already have the TopologyTestDriver, I'd prefer to
> focus on keeping the mock lean. I agree it should be in the javadoc as well
> as the web documentation.
> 
> Thanks,
> -John
> 
> On Thu, Mar 8, 2018 at 1:46 PM, Guozhang Wang <wangg...@gmail.com> wrote:
> 
>> Hello John,
>>
>> Thanks for the KIP. I made a pass over the wiki page and here are some
>> comments:
>>
>> 1. Meta-comment: there is an internal class MockProcessorContext under the
>> o.a.k.test package, which should be replaced as part of this KIP.
>>
>> 2. In @Override StreamsMetrics metrics(), will you return a fully created
>> StreamsMetricsImpl object or are you planning to use the
>> MockStreamsMetrics? Note that for the latter case you probably need to look
>> into https://issues.apache.org/jira/browse/KAFKA-5676 as well.
>>
>> 3. Not related to the KIP changes themselves: about
>> "context.scheduledPunctuators": we need to well document that in the
>> MockProcessorContext the scheduled punctuator will never by auto-triggered,
>> and hence it is only for testing people's code that some punctuators are
>> indeed registered, and if people want full auto punctuation testing they
>> have to go with TopologyTestDriver.
>>
>>
>>
>> Guozhang
>>
>>
>> On Wed, Mar 7, 2018 at 8:04 PM, John Roesler <j...@confluent.io> wrote:
>>
>>> On Wed, Mar 7, 2018 at 8:03 PM, John Roesler <j...@confluent.io> wrote:
>>>
>>>> Thanks Ted,
>>>>
>>>> Sure thing; I updated the example code in the KIP with a little
>> snippet.
>>>>
>>>> -John
>>>>
>>>> On Wed, Mar 7, 2018 at 7:18 PM, Ted Yu <yuzhih...@gmail.com> wrote:
>>>>
>>>>> Looks good.
>>>>>
>>>>> See if you can add punctuator into the sample code.
>>>>>
>>>>> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler <j...@confluent.io>
>> wrote:
>>>>>
>>>>>> Dear Kafka community,
>>>>>>
>>>>>> I am proposing KIP-267 to augment the public Streams test utils API.
>>>>>> The goal is to simplify testing of Kafka Streams applications.
>>>>>>
>>>>>> Please find details in the
>>>>>> wiki:https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>> 267%3A+Add+Processor+Unit+Test+Support+to+Kafka+Streams+Test+Utils
>>>>>>
>>>>>> An initial WIP PR can be found here:https://github.com/
>>>>>> apache/kafka/pull/4662
>>>>>>
>>>>>> I also included the user-list (please hit "reply-all" to include
>> both
>>>>>> lists in this KIP discussion).
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> -John
>>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>>
>>
>> --
>> -- Guozhang
>>
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to