Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-09 Thread John Roesler
Sweet! I think this pretty much wraps up all the discussion points.

I'll update the KIP with all the relevant aspects we discussed and call for
a vote.

I'll also comment on the TopologyTestDriver ticket noting this modular test
strategy.

Thanks, everyone.
-John

On Fri, Mar 9, 2018 at 10:57 AM, Guozhang Wang  wrote:

> Hey John,
>
> Re: Mock Processor Context:
>
> That's a good point, I'm convinced that we should keep them as two classes.
>
>
> Re: test-utils module:
>
> I think I agree with your proposed changes, in fact in order to not scatter
> the test classes in two places maybe it's better to move all of them to the
> new module. One caveat is that it will make streams' project hierarchy
> inconsistent with other projects where the unit test classes are maintained
> inside the main artifact package, but I think it is a good cost to pay,
> plus once we start publishing test-util artifacts for other projects like
> client and connect, we may face the same issue and need to do this
> refactoring as well.
>
>
>
> Guozhang
>
>
>
>
> On Fri, Mar 9, 2018 at 9:54 AM, John Roesler  wrote:
>
> > Hi Guozhang and Bill,
> >
> > I'll summarize what I'm currently thinking in light of all the
> discussion:
> >
> > Mock Processor Context:
> > ===
> >
> > Here's how I see the use cases for the two mocks differing:
> >
> > 1. o.a.k.test.MPC: Crafted for testing Streams use cases. Implements
> > AbstractProcessorContext, actually forward to child processor nodes,
> allow
> > restoring a state store. Most importantly, the freedom to do stuff
> > convenient for our tests without impacting anyone.
> >
> > 2. (test-utils) MPC: Crafted for testing community Processors (and
> > friends). Very flat and simple implementation (so people can read it in
> one
> > sitting); i.e., doesn't drag in other data models like RecordContext.
> Test
> > one processor in isolation, so generally don't bother with complex logic
> > like scheduling punctuators, forwarding results, or restoring state
> stores.
> > Most importantly, an API that can be stable.
> >
> > So, I really am leaning toward keeping both implementations. I like
> Bill's
> > suggestion of renaming the unit testing class to
> > InternalMockProcessorContext, since having classes with the same name in
> > different packages is confusing. I look forward to the day when Java 9
> > takes off and we can actually hide internal classes from the public
> > interface.
> >
> > test-utils module:
> > =
> >
> > This is actually out of scope for this KIP if we keep both MPC
> > implementations, but it has been a major feature of this discussion, so
> we
> > may as well see it though.
> >
> > I've waffled a bit on this point, but right now I would propose we
> > restructure the streams directory thusly:
> >
> > streams/ (artifact name := "streams", the actual streams code lives here)
> > - test-utils/ (this is the current test-utils artifact, depends on
> > "streams")
> > - tests/ (new module, depends on "streams" and "test-utils", *NO
> published
> > artifact*)
> >
> > This gets us out of the circular dependency without having to engage in
> any
> > Gradle shenanigans while preserving "test-utils" as a separate artifact.
> > This is good because: 1) the test-utils don't need to be in production
> > code, so it's nice to have a separate artifact, 2) test-utils is already
> > public in 1.1, and it's a bummer to introduce users' code when we can so
> > easily avoid it.
> >
> > Note, though, that if we agree to keep both MPC implementations, then
> this
> > really is just important for rewriting our tests to use
> TopologyTestDriver,
> > and in fact only the tests that need it should move to "streams/tests/".
> >
> > What say you?
> >
> > -John
> >
> > On Fri, Mar 9, 2018 at 9:01 AM, Guozhang Wang 
> wrote:
> >
> > > Hmm.. it seems to be a general issue then, since we were planning to
> also
> > > replace the KStreamTestDriver and ProcessorTopologyTestDriver with the
> > new
> > > TopologyTestDriver soon, so if the argument that testing dependency
> could
> > > still cause circular dependencies holds it means we cannot do that as
> > well.
> > >
> > > My understanding on gradle dependencies has been that test dependencies
> > are
> > > not required to compile when compiling the project, but only required
> > when
> > > testing the project; and the way we script gradle follows the way that
> > for
> > > any test tasks of the project we require compiling it first so this is
> > > fine. John / Bill, could you elaborate a bit more on the maintenance
> > > complexity concerns?
> > >
> > >
> > > Guozhang
> > >
> > > On Fri, Mar 9, 2018 at 7:40 AM, Bill Bejeck  wrote:
> > >
> > > > John,
> > > >
> > > > Sorry for the delayed response.  Thanks for the KIP, I'm +1 on it,
> and
> > I
> > > > don't have any further comments on the KIP itself aside from the
> > comments
> > > > that others have 

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-09 Thread Bill Bejeck
John,

Sorry for the delayed response.  Thanks for the KIP, I'm +1 on it, and I
don't have any further comments on the KIP itself aside from the comments
that others have raised.

Regarding the existing MockProcessorContext and its removal in favor of the
one added from this KIP, I'm actually in favor of keeping both.

IMHO it's reasonable to have both because the testing requirements are
different.  Most users are trying to verify their logic works as expected
within a Kafka Streams application and aren't concerned (or shouldn't be at
least, again IMHO) with testing Kafka Streams itself, that is the
responsibility of the Kafka Streams developers and contributors.

However, for the developers and contributors of Kafka Streams, the need to
test the internals of how Streams works is the primary concern and could at
times require different logic or available methods from a given mock object.

I have a couple of thoughts on mitigation of having two
MockProcessorContext objects

   1. Leave the current MockProcessorContext in the o.a.k.test package but
   rename it to InternalMockProcessorContext and add some documentation as to
   why it's there.
   2. Create a new package under o.a.k.test, called internals and move the
   existing MockProcessorContext there, but that would require a change to the
   visibility of the MockProcessorContext#allStateStores() method to public.

Just wanted to throw in my 2 cents.

Thanks,
Bill

On Thu, Mar 8, 2018 at 11:51 PM, John Roesler  wrote:

> I think what you're suggesting is to:
> 1. compile the main streams code, but not the tests
> 2. compile test-utils (and compile and run the test-utils tests)
> 3. compile and run the streams tests
>
> This works in theory, since the test-utils depends on the main streams
> code, but not the streams tests. and the streams tests depend on test-utils
> while the main streams code does not.
>
> But after poking around a bit and reading up on it, I think this is not
> possible, or at least not mainstream.
>
> The issue is that dependencies are formed between projects, in this case
> streams and streams:test-utils. The upstream project must be built before
> the dependant one, regardless of whether the dependency is for compiling
> the main code or the test code. This means we do have a circular dependency
> on our hands if we want the tests in streams to use the test-utils, since
> they'd both have to be built before the other.
>
> Gradle seems to be quite scriptable, so there may be some way to achieve
> this, but increasing the complexity of the build also introduces a project
> maintenance concern.
>
> The MockProcessorContext itself is pretty simple, so I'm tempted to argue
> that we should just have one for internal unit tests and another for
> test-utils, however this situation also afflicts KAFKA-6474
> , and the
> TopologyTestDriver is not so trivial.
>
> I think the best thing at this point is to go ahead and fold the test-utils
> into the streams project. We can put it into a separate "testutils" package
> to make it easy to identify which code is for test support and which code
> is Kafka Streams. The biggest bummer about this suggestion is that it we
> *just* introduced the test-utils artifact, so folks would to add that
> artifact in 1.1 to write their tests and then have to drop it again in 1.2.
>
> The other major solution is to create a new gradle project for the streams
> unit tests, which depends on streams and test-utils and move all the
> streams unit tests there. I'm pretty sure we can configure gradle just to
> include this project for running tests and not actually package any
> artifacts. This structure basically expresses your observation that the
> test code is essentially a separate module from the main streams code.
>
> Of course, I'm open to alternatives, especially if someone with more
> experience in Gradle is aware of a solution.
>
> Thanks,
> -John
>
>
> On Thu, Mar 8, 2018 at 3:39 PM, Matthias J. Sax 
> wrote:
>
> > Isn't MockProcessorContext in o.a.k.test part of the unit-test package
> > but not the main package?
> >
> > This should resolve the dependency issue.
> >
> > -Matthias
> >
> > On 3/8/18 3:32 PM, John Roesler wrote:
> > > Actually, replacing the MockProcessorContext in o.a.k.test could be a
> bit
> > > tricky, since it would make the "streams" module depend on
> > > "streams:test-utils", but "streams:test-utils" already depends on
> > "streams".
> > >
> > > At first glance, it seems like the options are:
> > > 1. leave the two separate implementations in place. This shouldn't be
> > > underestimated, especially since our internal tests may need different
> > > things from a mocked P.C. than our API users.
> > > 2. move the public testing artifacts into the regular streams module
> > > 3. move the unit tests for Streams into a third module that depends on
> > both
> > > streams and test-utils. Yuck!
> > 

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
I think what you're suggesting is to:
1. compile the main streams code, but not the tests
2. compile test-utils (and compile and run the test-utils tests)
3. compile and run the streams tests

This works in theory, since the test-utils depends on the main streams
code, but not the streams tests. and the streams tests depend on test-utils
while the main streams code does not.

But after poking around a bit and reading up on it, I think this is not
possible, or at least not mainstream.

The issue is that dependencies are formed between projects, in this case
streams and streams:test-utils. The upstream project must be built before
the dependant one, regardless of whether the dependency is for compiling
the main code or the test code. This means we do have a circular dependency
on our hands if we want the tests in streams to use the test-utils, since
they'd both have to be built before the other.

Gradle seems to be quite scriptable, so there may be some way to achieve
this, but increasing the complexity of the build also introduces a project
maintenance concern.

The MockProcessorContext itself is pretty simple, so I'm tempted to argue
that we should just have one for internal unit tests and another for
test-utils, however this situation also afflicts KAFKA-6474
, and the
TopologyTestDriver is not so trivial.

I think the best thing at this point is to go ahead and fold the test-utils
into the streams project. We can put it into a separate "testutils" package
to make it easy to identify which code is for test support and which code
is Kafka Streams. The biggest bummer about this suggestion is that it we
*just* introduced the test-utils artifact, so folks would to add that
artifact in 1.1 to write their tests and then have to drop it again in 1.2.

The other major solution is to create a new gradle project for the streams
unit tests, which depends on streams and test-utils and move all the
streams unit tests there. I'm pretty sure we can configure gradle just to
include this project for running tests and not actually package any
artifacts. This structure basically expresses your observation that the
test code is essentially a separate module from the main streams code.

Of course, I'm open to alternatives, especially if someone with more
experience in Gradle is aware of a solution.

Thanks,
-John


On Thu, Mar 8, 2018 at 3:39 PM, Matthias J. Sax 
wrote:

> Isn't MockProcessorContext in o.a.k.test part of the unit-test package
> but not the main package?
>
> This should resolve the dependency issue.
>
> -Matthias
>
> On 3/8/18 3:32 PM, John Roesler wrote:
> > Actually, replacing the MockProcessorContext in o.a.k.test could be a bit
> > tricky, since it would make the "streams" module depend on
> > "streams:test-utils", but "streams:test-utils" already depends on
> "streams".
> >
> > At first glance, it seems like the options are:
> > 1. leave the two separate implementations in place. This shouldn't be
> > underestimated, especially since our internal tests may need different
> > things from a mocked P.C. than our API users.
> > 2. move the public testing artifacts into the regular streams module
> > 3. move the unit tests for Streams into a third module that depends on
> both
> > streams and test-utils. Yuck!
> >
> > Thanks,
> > -John
> >
> > On Thu, Mar 8, 2018 at 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 
> 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
> 

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread Guozhang Wang
MockProcessorContext is only used in unit tests, and hence we should be
able to declare it as a test dependency of `streams` in gradle build file,
which is OK.


Guozhang

On Thu, Mar 8, 2018 at 3:32 PM, John Roesler  wrote:

> Actually, replacing the MockProcessorContext in o.a.k.test could be a bit
> tricky, since it would make the "streams" module depend on
> "streams:test-utils", but "streams:test-utils" already depends on
> "streams".
>
> At first glance, it seems like the options are:
> 1. leave the two separate implementations in place. This shouldn't be
> underestimated, especially since our internal tests may need different
> things from a mocked P.C. than our API users.
> 2. move the public testing artifacts into the regular streams module
> 3. move the unit tests for Streams into a third module that depends on both
> streams and test-utils. Yuck!
>
> Thanks,
> -John
>
> On Thu, Mar 8, 2018 at 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 
> 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  wrote:
> >>
> >> > On Wed, Mar 7, 2018 at 8:03 PM, John Roesler 
> 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  wrote:
> >> > >
> >> > >> Looks good.
> >> > >>
> >> > >> See if you can add punctuator into the sample code.
> >> > >>
> >> > >> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler 
> >> 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
> >>
> >
> >
>



-- 
-- Guozhang


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
Thanks, Matthias,

1. I can move it into the o.a.k.streams.processor package; that makes sense.

2. I'm expecting most users to use in-memory state stores, so they won't
need a state directory. In the "real" code path, the stateDir is extracted
from the config
by org.apache.kafka.streams.processor.internals.StateDirectory. The logic
is non-trivial and invoking it directly will result in the state directory
actually being created. Given my assumption that you don't need it most of
the time, creating directories seems too heavy to me.

3. I'm on the fence about that. It's not too much trouble to implement it,
even if it is deprecated from day 1, so I think I'd rather put it in and
let us remove it later when we actually remove the deprecated method. In
contrast, we actually would have to jump through some hoops to support
schedule(interval).

On Thu, Mar 8, 2018 at 3:36 PM, Matthias J. Sax 
wrote:

> 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 
> 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  wrote:
> >>
> >>> On Wed, Mar 7, 2018 at 8:03 PM, John Roesler 
> 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  wrote:
> 
> > Looks good.
> >
> > See if you can add punctuator into the sample code.
> >
> > On Wed, Mar 7, 2018 at 7:10 PM, John Roesler 
> >> 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
> >>
> >
> 
> 
> >>>
> >>
> >>
> >>
> >> --
> >> -- 

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread Matthias J. Sax
Isn't MockProcessorContext in o.a.k.test part of the unit-test package
but not the main package?

This should resolve the dependency issue.

-Matthias

On 3/8/18 3:32 PM, John Roesler wrote:
> Actually, replacing the MockProcessorContext in o.a.k.test could be a bit
> tricky, since it would make the "streams" module depend on
> "streams:test-utils", but "streams:test-utils" already depends on "streams".
> 
> At first glance, it seems like the options are:
> 1. leave the two separate implementations in place. This shouldn't be
> underestimated, especially since our internal tests may need different
> things from a mocked P.C. than our API users.
> 2. move the public testing artifacts into the regular streams module
> 3. move the unit tests for Streams into a third module that depends on both
> streams and test-utils. Yuck!
> 
> Thanks,
> -John
> 
> On Thu, Mar 8, 2018 at 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  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  wrote:
>>>
 On Wed, Mar 7, 2018 at 8:03 PM, John Roesler  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  wrote:
>
>> Looks good.
>>
>> See if you can add punctuator into the sample code.
>>
>> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler 
>>> 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
>>>
>>
>>
> 



signature.asc
Description: OpenPGP digital signature


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread Matthias J. Sax
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  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  wrote:
>>
>>> On Wed, Mar 7, 2018 at 8:03 PM, John Roesler  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  wrote:

> Looks good.
>
> See if you can add punctuator into the sample code.
>
> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler 
>> 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
>>
> 



signature.asc
Description: OpenPGP digital signature


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
Actually, replacing the MockProcessorContext in o.a.k.test could be a bit
tricky, since it would make the "streams" module depend on
"streams:test-utils", but "streams:test-utils" already depends on "streams".

At first glance, it seems like the options are:
1. leave the two separate implementations in place. This shouldn't be
underestimated, especially since our internal tests may need different
things from a mocked P.C. than our API users.
2. move the public testing artifacts into the regular streams module
3. move the unit tests for Streams into a third module that depends on both
streams and test-utils. Yuck!

Thanks,
-John

On Thu, Mar 8, 2018 at 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  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  wrote:
>>
>> > On Wed, Mar 7, 2018 at 8:03 PM, John Roesler  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  wrote:
>> > >
>> > >> Looks good.
>> > >>
>> > >> See if you can add punctuator into the sample code.
>> > >>
>> > >> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler 
>> 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
>>
>
>


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
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  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  wrote:
>
> > On Wed, Mar 7, 2018 at 8:03 PM, John Roesler  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  wrote:
> > >
> > >> Looks good.
> > >>
> > >> See if you can add punctuator into the sample code.
> > >>
> > >> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler 
> 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
>


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-07 Thread John Roesler
On Wed, Mar 7, 2018 at 8:03 PM, John Roesler  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  wrote:
>
>> Looks good.
>>
>> See if you can add punctuator into the sample code.
>>
>> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler  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
>> >
>>
>
>


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-07 Thread John Roesler
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  wrote:

> Looks good.
>
> See if you can add punctuator into the sample code.
>
> On Wed, Mar 7, 2018 at 7:10 PM, John Roesler  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
> >
>


Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-07 Thread Ted Yu
Looks good.

See if you can add punctuator into the sample code.

On Wed, Mar 7, 2018 at 7:10 PM, John Roesler  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
>