You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Ali Akhtar <al...@gmail.com> on 2016/11/11 14:28:45 UTC

Unit tests (in Java) take a *very* long time to 'clean up'?

I have some unit tests in which I create an embedded single broker kafka
cluster, using :

EmbeddedSingleNodeKafkaCluster.java from
https://github.com/confluentinc/examples/blob/master/kafka-streams/src/test/java/io/confluent/examples/streams/kafka/EmbeddedSingleNodeKafkaCluster.java

That class also creates an embedded zookeeper cluster / instance.

The problem is, while the tests run pretty fast and pass, they then stay
stuck in the 'teardown / clean up' stage for a really long time, often upto
10-20
seconds per test.

As I have a lot of test classes, each class creating its own embedded kafka
cluster, this time can really add up during compiles.

Is it possible to get these test classes to not do any clean up / safety
stuff, because the instances are just throwaway. Just have them kill -9 the
kafka / zookeeper and exit?

It doesn't make any sense that tests pass within seconds, but can't move on
to the next test class because its cleaning up.

I also have an embedded cassandra instance in these tests, but I don't
think that one is the problem, as i see a lot of zookeeper logs such as
these after the test runs:

133764 [main-SendThread(127.0.0.1:38846)] WARN
 org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for server
null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
at
org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)


Could it be that zookeeper doesn't exit and keeps retrying to connect?

Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Ali Akhtar <al...@gmail.com>.
For me, the startup doesn't take anywhere near as long as shutdown does.

On Fri, Nov 11, 2016 at 9:37 PM, Ali Akhtar <al...@gmail.com> wrote:

> Unless I'm missing anything, there's no reason why these throwaway
> processes should be shutdown gracefully. Just kill them as soon as the test
> finishes.
>
> On Fri, Nov 11, 2016 at 9:26 PM, Ali Akhtar <al...@gmail.com> wrote:
>
>> Hey Eno,
>>
>> Thanks for the quick reply.
>>
>> In the meantime, is it possible to just send a sigterm / kill -9 which
>> just kills the zookeeper + kafka? I can figure out how to do it if you can
>> point out which class / method creates the processes / threads.
>>
>> Thanks.
>>
>> On Fri, Nov 11, 2016 at 9:24 PM, Eno Thereska <en...@gmail.com>
>> wrote:
>>
>>> Hi Ali,
>>>
>>> You're right, shutting down the broker and ZK is expensive. We kept the
>>> number of integration tests relatively small (and pushed some more tests as
>>> system tests, while doing as many as possible as unit tests). It's not just
>>> the shutdown that's expensive, it's also the starting up unfortunately.
>>> It's on our todo list to do something about this, but we haven't gotten
>>> there yet. If someone from the community wants to have a look and help out,
>>> that'd be great (with a JIRA and PR).
>>>
>>> About the second problem with ZK logs, this is being worked on as part
>>> of removing the ZK dependency from streams and should be merged shortly:
>>> https://github.com/apache/kafka/pull/1884 <https://github.com/apache/kaf
>>> ka/pull/1884>. The msg you see does not affect correctness, it's just
>>> annoying and it will go away.
>>>
>>> Thanks,
>>> Eno
>>>
>>>
>>> > On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
>>> >
>>> > I have some unit tests in which I create an embedded single broker
>>> kafka
>>> > cluster, using :
>>> >
>>> > EmbeddedSingleNodeKafkaCluster.java from
>>> > https://github.com/confluentinc/examples/blob/master/kafka-s
>>> treams/src/test/java/io/confluent/examples/streams/kafka/
>>> EmbeddedSingleNodeKafkaCluster.java
>>> >
>>> > That class also creates an embedded zookeeper cluster / instance.
>>> >
>>> > The problem is, while the tests run pretty fast and pass, they then
>>> stay
>>> > stuck in the 'teardown / clean up' stage for a really long time, often
>>> upto
>>> > 10-20
>>> > seconds per test.
>>> >
>>> > As I have a lot of test classes, each class creating its own embedded
>>> kafka
>>> > cluster, this time can really add up during compiles.
>>> >
>>> > Is it possible to get these test classes to not do any clean up /
>>> safety
>>> > stuff, because the instances are just throwaway. Just have them kill
>>> -9 the
>>> > kafka / zookeeper and exit?
>>> >
>>> > It doesn't make any sense that tests pass within seconds, but can't
>>> move on
>>> > to the next test class because its cleaning up.
>>> >
>>> > I also have an embedded cassandra instance in these tests, but I don't
>>> > think that one is the problem, as i see a lot of zookeeper logs such as
>>> > these after the test runs:
>>> >
>>> > 133764 [main-SendThread(127.0.0.1:38846)] WARN
>>> > org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for
>>> server
>>> > null, unexpected error, closing socket connection and attempting
>>> reconnect
>>> > java.net.ConnectException: Connection refused
>>> > at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>>> > at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl
>>> .java:717)
>>> > at
>>> > org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientC
>>> nxnSocketNIO.java:361)
>>> > at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.ja
>>> va:1081)
>>> >
>>> >
>>> > Could it be that zookeeper doesn't exit and keeps retrying to connect?
>>>
>>>
>>
>

Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Ali Akhtar <al...@gmail.com>.
Unless I'm missing anything, there's no reason why these throwaway
processes should be shutdown gracefully. Just kill them as soon as the test
finishes.

On Fri, Nov 11, 2016 at 9:26 PM, Ali Akhtar <al...@gmail.com> wrote:

> Hey Eno,
>
> Thanks for the quick reply.
>
> In the meantime, is it possible to just send a sigterm / kill -9 which
> just kills the zookeeper + kafka? I can figure out how to do it if you can
> point out which class / method creates the processes / threads.
>
> Thanks.
>
> On Fri, Nov 11, 2016 at 9:24 PM, Eno Thereska <en...@gmail.com>
> wrote:
>
>> Hi Ali,
>>
>> You're right, shutting down the broker and ZK is expensive. We kept the
>> number of integration tests relatively small (and pushed some more tests as
>> system tests, while doing as many as possible as unit tests). It's not just
>> the shutdown that's expensive, it's also the starting up unfortunately.
>> It's on our todo list to do something about this, but we haven't gotten
>> there yet. If someone from the community wants to have a look and help out,
>> that'd be great (with a JIRA and PR).
>>
>> About the second problem with ZK logs, this is being worked on as part of
>> removing the ZK dependency from streams and should be merged shortly:
>> https://github.com/apache/kafka/pull/1884 <https://github.com/apache/kaf
>> ka/pull/1884>. The msg you see does not affect correctness, it's just
>> annoying and it will go away.
>>
>> Thanks,
>> Eno
>>
>>
>> > On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
>> >
>> > I have some unit tests in which I create an embedded single broker kafka
>> > cluster, using :
>> >
>> > EmbeddedSingleNodeKafkaCluster.java from
>> > https://github.com/confluentinc/examples/blob/master/kafka-
>> streams/src/test/java/io/confluent/examples/streams/
>> kafka/EmbeddedSingleNodeKafkaCluster.java
>> >
>> > That class also creates an embedded zookeeper cluster / instance.
>> >
>> > The problem is, while the tests run pretty fast and pass, they then stay
>> > stuck in the 'teardown / clean up' stage for a really long time, often
>> upto
>> > 10-20
>> > seconds per test.
>> >
>> > As I have a lot of test classes, each class creating its own embedded
>> kafka
>> > cluster, this time can really add up during compiles.
>> >
>> > Is it possible to get these test classes to not do any clean up / safety
>> > stuff, because the instances are just throwaway. Just have them kill -9
>> the
>> > kafka / zookeeper and exit?
>> >
>> > It doesn't make any sense that tests pass within seconds, but can't
>> move on
>> > to the next test class because its cleaning up.
>> >
>> > I also have an embedded cassandra instance in these tests, but I don't
>> > think that one is the problem, as i see a lot of zookeeper logs such as
>> > these after the test runs:
>> >
>> > 133764 [main-SendThread(127.0.0.1:38846)] WARN
>> > org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for server
>> > null, unexpected error, closing socket connection and attempting
>> reconnect
>> > java.net.ConnectException: Connection refused
>> > at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>> > at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl
>> .java:717)
>> > at
>> > org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientC
>> nxnSocketNIO.java:361)
>> > at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
>> >
>> >
>> > Could it be that zookeeper doesn't exit and keeps retrying to connect?
>>
>>
>

Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Ali Akhtar <al...@gmail.com>.
Eno,

I tried the following, but i haven't seen any improvement. In fact it seems
to run longer. Are you sure Kafka / ZK are run as threads and not as
process:


        Set<Thread> threadSet = Thread.getAllStackTraces().keySet();
        threadSet.forEach(t ->
        {
            if (t == Thread.currentThread())
                return;

            t.stop();
        });


On Fri, Nov 11, 2016 at 9:52 PM, Ali Akhtar <al...@gmail.com> wrote:

> Oh, so it seems like there's no easy way to just Thread.stop() without
> changing the internal kafka / zk code? :(
>
> Perhaps its possible to start kafka / zk within another thread, and then
> kill the wrapper thread. Will that stop the children threads, if the
> wrapper thread is killed?
>
> Hmm, or may be a an Executor which is shutdown, and which force shuts down
> the children threads?
>
>
>
> On Fri, Nov 11, 2016 at 9:46 PM, Eno Thereska <en...@gmail.com>
> wrote:
>
>> It's the org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster
>> (start ZK and calls org.apache.kafka.streams.integration.utils.KafkaEmbedded
>> to start Kafka).
>> So these are embedded in the sense that it's not another process, just
>> threads within the main streams test process.
>>
>> Thanks
>> Eno
>>
>> > On 11 Nov 2016, at 16:26, Ali Akhtar <al...@gmail.com> wrote:
>> >
>> > Hey Eno,
>> >
>> > Thanks for the quick reply.
>> >
>> > In the meantime, is it possible to just send a sigterm / kill -9 which
>> just
>> > kills the zookeeper + kafka? I can figure out how to do it if you can
>> point
>> > out which class / method creates the processes / threads.
>> >
>> > Thanks.
>> >
>> > On Fri, Nov 11, 2016 at 9:24 PM, Eno Thereska <en...@gmail.com>
>> > wrote:
>> >
>> >> Hi Ali,
>> >>
>> >> You're right, shutting down the broker and ZK is expensive. We kept the
>> >> number of integration tests relatively small (and pushed some more
>> tests as
>> >> system tests, while doing as many as possible as unit tests). It's not
>> just
>> >> the shutdown that's expensive, it's also the starting up unfortunately.
>> >> It's on our todo list to do something about this, but we haven't gotten
>> >> there yet. If someone from the community wants to have a look and help
>> out,
>> >> that'd be great (with a JIRA and PR).
>> >>
>> >> About the second problem with ZK logs, this is being worked on as part
>> of
>> >> removing the ZK dependency from streams and should be merged shortly:
>> >> https://github.com/apache/kafka/pull/1884 <https://github.com/apache/
>> >> kafka/pull/1884>. The msg you see does not affect correctness, it's
>> just
>> >> annoying and it will go away.
>> >>
>> >> Thanks,
>> >> Eno
>> >>
>> >>
>> >>> On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
>> >>>
>> >>> I have some unit tests in which I create an embedded single broker
>> kafka
>> >>> cluster, using :
>> >>>
>> >>> EmbeddedSingleNodeKafkaCluster.java from
>> >>> https://github.com/confluentinc/examples/blob/
>> >> master/kafka-streams/src/test/java/io/confluent/examples/str
>> eams/kafka/
>> >> EmbeddedSingleNodeKafkaCluster.java
>> >>>
>> >>> That class also creates an embedded zookeeper cluster / instance.
>> >>>
>> >>> The problem is, while the tests run pretty fast and pass, they then
>> stay
>> >>> stuck in the 'teardown / clean up' stage for a really long time, often
>> >> upto
>> >>> 10-20
>> >>> seconds per test.
>> >>>
>> >>> As I have a lot of test classes, each class creating its own embedded
>> >> kafka
>> >>> cluster, this time can really add up during compiles.
>> >>>
>> >>> Is it possible to get these test classes to not do any clean up /
>> safety
>> >>> stuff, because the instances are just throwaway. Just have them kill
>> -9
>> >> the
>> >>> kafka / zookeeper and exit?
>> >>>
>> >>> It doesn't make any sense that tests pass within seconds, but can't
>> move
>> >> on
>> >>> to the next test class because its cleaning up.
>> >>>
>> >>> I also have an embedded cassandra instance in these tests, but I don't
>> >>> think that one is the problem, as i see a lot of zookeeper logs such
>> as
>> >>> these after the test runs:
>> >>>
>> >>> 133764 [main-SendThread(127.0.0.1:38846)] WARN
>> >>> org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for
>> server
>> >>> null, unexpected error, closing socket connection and attempting
>> >> reconnect
>> >>> java.net.ConnectException: Connection refused
>> >>> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>> >>> at sun.nio.ch.SocketChannelImpl.finishConnect(
>> >> SocketChannelImpl.java:717)
>> >>> at
>> >>> org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
>> >> ClientCnxnSocketNIO.java:361)
>> >>> at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.
>> java:1081)
>> >>>
>> >>>
>> >>> Could it be that zookeeper doesn't exit and keeps retrying to connect?
>> >>
>> >>
>>
>>
>

Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Ali Akhtar <al...@gmail.com>.
Oh, so it seems like there's no easy way to just Thread.stop() without
changing the internal kafka / zk code? :(

Perhaps its possible to start kafka / zk within another thread, and then
kill the wrapper thread. Will that stop the children threads, if the
wrapper thread is killed?

Hmm, or may be a an Executor which is shutdown, and which force shuts down
the children threads?



On Fri, Nov 11, 2016 at 9:46 PM, Eno Thereska <en...@gmail.com>
wrote:

> It's the org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster
> (start ZK and calls org.apache.kafka.streams.integration.utils.KafkaEmbedded
> to start Kafka).
> So these are embedded in the sense that it's not another process, just
> threads within the main streams test process.
>
> Thanks
> Eno
>
> > On 11 Nov 2016, at 16:26, Ali Akhtar <al...@gmail.com> wrote:
> >
> > Hey Eno,
> >
> > Thanks for the quick reply.
> >
> > In the meantime, is it possible to just send a sigterm / kill -9 which
> just
> > kills the zookeeper + kafka? I can figure out how to do it if you can
> point
> > out which class / method creates the processes / threads.
> >
> > Thanks.
> >
> > On Fri, Nov 11, 2016 at 9:24 PM, Eno Thereska <en...@gmail.com>
> > wrote:
> >
> >> Hi Ali,
> >>
> >> You're right, shutting down the broker and ZK is expensive. We kept the
> >> number of integration tests relatively small (and pushed some more
> tests as
> >> system tests, while doing as many as possible as unit tests). It's not
> just
> >> the shutdown that's expensive, it's also the starting up unfortunately.
> >> It's on our todo list to do something about this, but we haven't gotten
> >> there yet. If someone from the community wants to have a look and help
> out,
> >> that'd be great (with a JIRA and PR).
> >>
> >> About the second problem with ZK logs, this is being worked on as part
> of
> >> removing the ZK dependency from streams and should be merged shortly:
> >> https://github.com/apache/kafka/pull/1884 <https://github.com/apache/
> >> kafka/pull/1884>. The msg you see does not affect correctness, it's just
> >> annoying and it will go away.
> >>
> >> Thanks,
> >> Eno
> >>
> >>
> >>> On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
> >>>
> >>> I have some unit tests in which I create an embedded single broker
> kafka
> >>> cluster, using :
> >>>
> >>> EmbeddedSingleNodeKafkaCluster.java from
> >>> https://github.com/confluentinc/examples/blob/
> >> master/kafka-streams/src/test/java/io/confluent/examples/streams/kafka/
> >> EmbeddedSingleNodeKafkaCluster.java
> >>>
> >>> That class also creates an embedded zookeeper cluster / instance.
> >>>
> >>> The problem is, while the tests run pretty fast and pass, they then
> stay
> >>> stuck in the 'teardown / clean up' stage for a really long time, often
> >> upto
> >>> 10-20
> >>> seconds per test.
> >>>
> >>> As I have a lot of test classes, each class creating its own embedded
> >> kafka
> >>> cluster, this time can really add up during compiles.
> >>>
> >>> Is it possible to get these test classes to not do any clean up /
> safety
> >>> stuff, because the instances are just throwaway. Just have them kill -9
> >> the
> >>> kafka / zookeeper and exit?
> >>>
> >>> It doesn't make any sense that tests pass within seconds, but can't
> move
> >> on
> >>> to the next test class because its cleaning up.
> >>>
> >>> I also have an embedded cassandra instance in these tests, but I don't
> >>> think that one is the problem, as i see a lot of zookeeper logs such as
> >>> these after the test runs:
> >>>
> >>> 133764 [main-SendThread(127.0.0.1:38846)] WARN
> >>> org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for
> server
> >>> null, unexpected error, closing socket connection and attempting
> >> reconnect
> >>> java.net.ConnectException: Connection refused
> >>> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> >>> at sun.nio.ch.SocketChannelImpl.finishConnect(
> >> SocketChannelImpl.java:717)
> >>> at
> >>> org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
> >> ClientCnxnSocketNIO.java:361)
> >>> at org.apache.zookeeper.ClientCnxn$SendThread.run(
> ClientCnxn.java:1081)
> >>>
> >>>
> >>> Could it be that zookeeper doesn't exit and keeps retrying to connect?
> >>
> >>
>
>

Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Eno Thereska <en...@gmail.com>.
It's the org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster (start ZK and calls org.apache.kafka.streams.integration.utils.KafkaEmbedded to start Kafka).
So these are embedded in the sense that it's not another process, just threads within the main streams test process.

Thanks
Eno

> On 11 Nov 2016, at 16:26, Ali Akhtar <al...@gmail.com> wrote:
> 
> Hey Eno,
> 
> Thanks for the quick reply.
> 
> In the meantime, is it possible to just send a sigterm / kill -9 which just
> kills the zookeeper + kafka? I can figure out how to do it if you can point
> out which class / method creates the processes / threads.
> 
> Thanks.
> 
> On Fri, Nov 11, 2016 at 9:24 PM, Eno Thereska <en...@gmail.com>
> wrote:
> 
>> Hi Ali,
>> 
>> You're right, shutting down the broker and ZK is expensive. We kept the
>> number of integration tests relatively small (and pushed some more tests as
>> system tests, while doing as many as possible as unit tests). It's not just
>> the shutdown that's expensive, it's also the starting up unfortunately.
>> It's on our todo list to do something about this, but we haven't gotten
>> there yet. If someone from the community wants to have a look and help out,
>> that'd be great (with a JIRA and PR).
>> 
>> About the second problem with ZK logs, this is being worked on as part of
>> removing the ZK dependency from streams and should be merged shortly:
>> https://github.com/apache/kafka/pull/1884 <https://github.com/apache/
>> kafka/pull/1884>. The msg you see does not affect correctness, it's just
>> annoying and it will go away.
>> 
>> Thanks,
>> Eno
>> 
>> 
>>> On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
>>> 
>>> I have some unit tests in which I create an embedded single broker kafka
>>> cluster, using :
>>> 
>>> EmbeddedSingleNodeKafkaCluster.java from
>>> https://github.com/confluentinc/examples/blob/
>> master/kafka-streams/src/test/java/io/confluent/examples/streams/kafka/
>> EmbeddedSingleNodeKafkaCluster.java
>>> 
>>> That class also creates an embedded zookeeper cluster / instance.
>>> 
>>> The problem is, while the tests run pretty fast and pass, they then stay
>>> stuck in the 'teardown / clean up' stage for a really long time, often
>> upto
>>> 10-20
>>> seconds per test.
>>> 
>>> As I have a lot of test classes, each class creating its own embedded
>> kafka
>>> cluster, this time can really add up during compiles.
>>> 
>>> Is it possible to get these test classes to not do any clean up / safety
>>> stuff, because the instances are just throwaway. Just have them kill -9
>> the
>>> kafka / zookeeper and exit?
>>> 
>>> It doesn't make any sense that tests pass within seconds, but can't move
>> on
>>> to the next test class because its cleaning up.
>>> 
>>> I also have an embedded cassandra instance in these tests, but I don't
>>> think that one is the problem, as i see a lot of zookeeper logs such as
>>> these after the test runs:
>>> 
>>> 133764 [main-SendThread(127.0.0.1:38846)] WARN
>>> org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for server
>>> null, unexpected error, closing socket connection and attempting
>> reconnect
>>> java.net.ConnectException: Connection refused
>>> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>>> at sun.nio.ch.SocketChannelImpl.finishConnect(
>> SocketChannelImpl.java:717)
>>> at
>>> org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
>> ClientCnxnSocketNIO.java:361)
>>> at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
>>> 
>>> 
>>> Could it be that zookeeper doesn't exit and keeps retrying to connect?
>> 
>> 


Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Ali Akhtar <al...@gmail.com>.
Hey Eno,

Thanks for the quick reply.

In the meantime, is it possible to just send a sigterm / kill -9 which just
kills the zookeeper + kafka? I can figure out how to do it if you can point
out which class / method creates the processes / threads.

Thanks.

On Fri, Nov 11, 2016 at 9:24 PM, Eno Thereska <en...@gmail.com>
wrote:

> Hi Ali,
>
> You're right, shutting down the broker and ZK is expensive. We kept the
> number of integration tests relatively small (and pushed some more tests as
> system tests, while doing as many as possible as unit tests). It's not just
> the shutdown that's expensive, it's also the starting up unfortunately.
> It's on our todo list to do something about this, but we haven't gotten
> there yet. If someone from the community wants to have a look and help out,
> that'd be great (with a JIRA and PR).
>
> About the second problem with ZK logs, this is being worked on as part of
> removing the ZK dependency from streams and should be merged shortly:
> https://github.com/apache/kafka/pull/1884 <https://github.com/apache/
> kafka/pull/1884>. The msg you see does not affect correctness, it's just
> annoying and it will go away.
>
> Thanks,
> Eno
>
>
> > On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
> >
> > I have some unit tests in which I create an embedded single broker kafka
> > cluster, using :
> >
> > EmbeddedSingleNodeKafkaCluster.java from
> > https://github.com/confluentinc/examples/blob/
> master/kafka-streams/src/test/java/io/confluent/examples/streams/kafka/
> EmbeddedSingleNodeKafkaCluster.java
> >
> > That class also creates an embedded zookeeper cluster / instance.
> >
> > The problem is, while the tests run pretty fast and pass, they then stay
> > stuck in the 'teardown / clean up' stage for a really long time, often
> upto
> > 10-20
> > seconds per test.
> >
> > As I have a lot of test classes, each class creating its own embedded
> kafka
> > cluster, this time can really add up during compiles.
> >
> > Is it possible to get these test classes to not do any clean up / safety
> > stuff, because the instances are just throwaway. Just have them kill -9
> the
> > kafka / zookeeper and exit?
> >
> > It doesn't make any sense that tests pass within seconds, but can't move
> on
> > to the next test class because its cleaning up.
> >
> > I also have an embedded cassandra instance in these tests, but I don't
> > think that one is the problem, as i see a lot of zookeeper logs such as
> > these after the test runs:
> >
> > 133764 [main-SendThread(127.0.0.1:38846)] WARN
> > org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for server
> > null, unexpected error, closing socket connection and attempting
> reconnect
> > java.net.ConnectException: Connection refused
> > at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> > at sun.nio.ch.SocketChannelImpl.finishConnect(
> SocketChannelImpl.java:717)
> > at
> > org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
> ClientCnxnSocketNIO.java:361)
> > at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
> >
> >
> > Could it be that zookeeper doesn't exit and keeps retrying to connect?
>
>

Re: Unit tests (in Java) take a *very* long time to 'clean up'?

Posted by Eno Thereska <en...@gmail.com>.
Hi Ali,

You're right, shutting down the broker and ZK is expensive. We kept the number of integration tests relatively small (and pushed some more tests as system tests, while doing as many as possible as unit tests). It's not just the shutdown that's expensive, it's also the starting up unfortunately. It's on our todo list to do something about this, but we haven't gotten there yet. If someone from the community wants to have a look and help out, that'd be great (with a JIRA and PR).

About the second problem with ZK logs, this is being worked on as part of removing the ZK dependency from streams and should be merged shortly: https://github.com/apache/kafka/pull/1884 <https://github.com/apache/kafka/pull/1884>. The msg you see does not affect correctness, it's just annoying and it will go away.

Thanks,
Eno
 

> On 11 Nov 2016, at 14:28, Ali Akhtar <al...@gmail.com> wrote:
> 
> I have some unit tests in which I create an embedded single broker kafka
> cluster, using :
> 
> EmbeddedSingleNodeKafkaCluster.java from
> https://github.com/confluentinc/examples/blob/master/kafka-streams/src/test/java/io/confluent/examples/streams/kafka/EmbeddedSingleNodeKafkaCluster.java
> 
> That class also creates an embedded zookeeper cluster / instance.
> 
> The problem is, while the tests run pretty fast and pass, they then stay
> stuck in the 'teardown / clean up' stage for a really long time, often upto
> 10-20
> seconds per test.
> 
> As I have a lot of test classes, each class creating its own embedded kafka
> cluster, this time can really add up during compiles.
> 
> Is it possible to get these test classes to not do any clean up / safety
> stuff, because the instances are just throwaway. Just have them kill -9 the
> kafka / zookeeper and exit?
> 
> It doesn't make any sense that tests pass within seconds, but can't move on
> to the next test class because its cleaning up.
> 
> I also have an embedded cassandra instance in these tests, but I don't
> think that one is the problem, as i see a lot of zookeeper logs such as
> these after the test runs:
> 
> 133764 [main-SendThread(127.0.0.1:38846)] WARN
> org.apache.zookeeper.ClientCnxn  - Session 0x15853c3497f0001 for server
> null, unexpected error, closing socket connection and attempting reconnect
> java.net.ConnectException: Connection refused
> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
> at
> org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
> at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
> 
> 
> Could it be that zookeeper doesn't exit and keeps retrying to connect?