Hello Rohit,

Thanks for picking this up! I think your KIP looks good.

While I was doing some cleanup of our tests before, one thing I encountered is 
that, while most tests don’t semantically need to specify any configs, many 
tests actually do set the state directory config. They set it specifically so 
that they can delete it at the end of the test. Otherwise, the tests would 
leave RocksDB directories behind.

I’m wondering if we should address this issue as part of your KIP. What I’m 
thinking is this: if no state directory is specified, then we create a new, 
unique temp directory and register it for cleanup when the JVM exits. 
Additionally, we would set a flag and clean up the state dir when TTD.close() 
is called.

That way, TTD tests would be by default independent and tidy.

Admittedly, this is outside the current scope of your KIP, so please feel free 
to reject this idea, in which case I can file a separate ticket for it. 

Thanks!
-John

On Tue, Nov 3, 2020, at 18:59, Rohit Deshpande wrote:
> Hi Matthias,
> Thank you for the review and the suggestion.
> Considering at most 3 parameters to the constructor of 
> TopologyTestDriver
> and topology being required parameter, we can definitely add a new
> constructor `TopologyTestDriver(Topology, Instant)` .
> Right now, I see one test where we can use this constructor:
> https://github.com/apache/kafka/blob/trunk/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java#L80-L83
> Also we can get rid of this method in TestDriver trait:
> https://github.com/apache/kafka/blob/trunk/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/TestDriver.scala#L32-L35
> which is used in multiple test classes and seems redundant. I agree with
> your suggestion.
> Thanks,
> Rohit
> 
> On Tue, Nov 3, 2020 at 3:19 PM Matthias J. Sax <mj...@apache.org> wrote:
> 
> > Thanks for the KIP Rohit.
> >
> > Wondering, if we should also add `TopologyTestDriver(Topology,
> > Instant)`? Not totally sure, as having too many overload could also be
> > annoying.
> >
> >
> > -Matthias
> >
> > On 11/3/20 10:02 AM, Rohit Deshpande wrote:
> > > Hello all,
> > > I have created KIP-680: TopologyTestDriver should not require a
> > Properties
> > > argument.
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-680%3A+TopologyTestDriver+should+not+require+a+Properties+argument
> > >
> > > Jira for the KIP:
> > > https://issues.apache.org/jira/browse/KAFKA-10629
> > >
> > > If we end up making changes, they will look like this:
> > > https://github.com/apache/kafka/compare/trunk...rohitrmd:KAFKA-10629
> > >
> > > Please have a look and let me know what you think.
> > >
> > > Thanks,
> > > Rohit
> > >
> >
>

Reply via email to