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,

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

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.

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

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

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

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

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

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

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

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

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

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

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