[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-4309?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354802#comment-17354802
 ] 

Francesco Nigro edited comment on ZOOKEEPER-4309 at 6/1/21, 7:19 AM:
---------------------------------------------------------------------

There are other parts in the zookeeper source code where a similar pattern is 
used and executor::shutdown is called to save the leak to happen.
The leak happen while halting QuorumCnxManager: listenerHandlers task can be 
already completed or are going to complete on halt, but the executorservice's 
thread are still alive, and given that not daemon threads are GC roots they 
won't be garbage collected, although not reacheable.
An ExecutorService which non-daemon threads are still alive prevent a main 
thread to complete.

An example of the existing code patterns that correctly shutdown the executor, 
you can find 
https://github.com/franz1981/zookeeper/blob/b4f9aab099880ba8ef08eaff697debe6cdeae057/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L456-L468


{code:java}
                ExecutorService executor = 
Executors.newFixedThreadPool(serverSockets.size());
                CountDownLatch latch = new CountDownLatch(serverSockets.size());

                serverSockets.forEach(serverSocket ->
                        executor.submit(new 
LearnerCnxAcceptorHandler(serverSocket, latch)));

                try {
                    latch.await();
                } catch (InterruptedException ie) {
                    LOG.error("Interrupted while sleeping in 
LearnerCnxAcceptor.", ie);
                } finally {
                    closeSockets();
                    executor.shutdown();
{code}

executor::shutdown is used to prevent the just created executor to leak, as 
explained above.

A sample program that shows this ExecutorService behaviour is:

{code:java}
    private static final boolean STOP_EXECUTOR = false;

    public static void main(String[] args) throws InterruptedException {
        final CountDownLatch interruptibleTask = new CountDownLatch(1);
        final CountDownLatch completed = new CountDownLatch(1);
        final CountDownLatch starting = new CountDownLatch(1);
        final ExecutorService service = Executors.newFixedThreadPool(1);
        service.submit(() -> {
            try {
                starting.countDown();
                interruptibleTask.await();
                System.err.println("Graceful stop");
            } catch (InterruptedException e) {
                System.err.println("This cannot happen");
            } finally {
                completed.countDown();
            }
        });
        if (STOP_EXECUTOR) {
            service.shutdown();
        }
        starting.await();
        System.err.println("Gracefully stop the submitted task");
        interruptibleTask.countDown();
        System.err.println("Await interruptible task to complete");
        completed.await();
        if (STOP_EXECUTOR) {
            System.err.println("The application can correctly complete and 
there is no leak");
        } else {
            System.err.println("Perform some full GC and take an heap dump: 
you'll se the service Thread still alive");
            System.err.println("And this application won't stop");
        }
    }
{code}

Setting STOP_EXECUTOR = false shows that the application never end because of 
the Thread leak, while STOP_EXECUTOR = true just make it works, and the 
application can exit with code 0.
Hope that's enough to explain what's the issue here.




was (Author: nigrofranz):
There are other parts in the zookeeper source code where a similar pattern is 
used and executor::shutdown is called to save the leak to happen.
The leak happen while halting QuorumCnxManager: listenerHandlers task can be 
already completed or are going to complete on halt, but the executorservice's 
thread are still alive, and given that are non-daemon, are GC roots and won't 
be garbage collected.
An ExecutorService which non-daemon threads are still alive prevent a main 
thread to complete.

An example of the existing code patterns that correctly shutdown the executor, 
you can find 
https://github.com/franz1981/zookeeper/blob/b4f9aab099880ba8ef08eaff697debe6cdeae057/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L456-L468


{code:java}
                ExecutorService executor = 
Executors.newFixedThreadPool(serverSockets.size());
                CountDownLatch latch = new CountDownLatch(serverSockets.size());

                serverSockets.forEach(serverSocket ->
                        executor.submit(new 
LearnerCnxAcceptorHandler(serverSocket, latch)));

                try {
                    latch.await();
                } catch (InterruptedException ie) {
                    LOG.error("Interrupted while sleeping in 
LearnerCnxAcceptor.", ie);
                } finally {
                    closeSockets();
                    executor.shutdown();
{code}

executor::shutdown is used to prevent the just created executor to leak, as 
explained above.

A sample program that shows this ExecutorService behaviour is:

{code:java}
    private static final boolean STOP_EXECUTOR = false;

    public static void main(String[] args) throws InterruptedException {
        final CountDownLatch interruptibleTask = new CountDownLatch(1);
        final CountDownLatch completed = new CountDownLatch(1);
        final CountDownLatch starting = new CountDownLatch(1);
        final ExecutorService service = Executors.newFixedThreadPool(1);
        service.submit(() -> {
            try {
                starting.countDown();
                interruptibleTask.await();
                System.err.println("Graceful stop");
            } catch (InterruptedException e) {
                System.err.println("This cannot happen");
            } finally {
                completed.countDown();
            }
        });
        if (STOP_EXECUTOR) {
            service.shutdown();
        }
        starting.await();
        System.err.println("Gracefully stop the submitted task");
        interruptibleTask.countDown();
        System.err.println("Await interruptible task to complete");
        completed.await();
        if (STOP_EXECUTOR) {
            System.err.println("The application can correctly complete and 
there is no leak");
        } else {
            System.err.println("Perform some full GC and take an heap dump: 
you'll se the service Thread still alive");
            System.err.println("And this application won't stop");
        }
    }
{code}

Setting STOP_EXECUTOR = false shows that the application never end because of 
the Thread leak, while STOP_EXECUTOR = true just make it works, and the 
application can exit with code 0.
Hope that's enough to explain what's the issue here.



> QuorumCnxManager's ListenerHandler thread leak
> ----------------------------------------------
>
>                 Key: ZOOKEEPER-4309
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4309
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: server
>    Affects Versions: 3.6.3, 3.7.0
>            Reporter: Francesco Nigro
>            Priority: Minor
>              Labels: pull-request-available
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> QuorumCnxManager::Listener::run is creating a 
> Executors.newFixedThreadPool(addresses.size()) without shutting it down after 
> ListenerHandler task has been completed causing it to leak.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to