You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by "Vinay (JIRA)" <ji...@apache.org> on 2013/06/07 06:15:19 UTC

[jira] [Commented] (BOOKKEEPER-313) Bookkeeper shutdown call from Bookie thread is not shutting down server

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

Vinay commented on BOOKKEEPER-313:
----------------------------------

This issue is very important to fix.
If Journal Thread exits due to some exception, then Bookie will not be shutdown. 
In the latest code, Bookie thread will wait at following location...
{code:java}        shutdownThread.start();
        try {
            shutdownThread.join();
        } catch (InterruptedException e) {
            Thread.currentThread().interrupt();
            LOG.debug("InterruptedException while waiting for shutdown. Not a problem!!");
        }{code}

There are two alternatives for this issue,, wither we can interrupt and join() Bookie thread, or remove join() call itself in {{Bookie#shutdown()}}. 
Any thoughts?
                
> Bookkeeper shutdown call from Bookie thread is not shutting down server
> -----------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-313
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-313
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-server
>    Affects Versions: 4.2.1
>            Reporter: Vinay
>
> shutdown(..) call from inside Bookie#run() is not shutting down the server.
> Bookie thread is waiting to join itself.
> Shutdown called from here
> {code} if (!shuttingdown) {
>             // some error found in journal thread and it quits
>             // following add operations to it would hang unit client timeout
>             // so we should let bookie server exists
>             LOG.error("Journal manager quits unexpectedly.");
>             shutdown(ExitCode.BOOKIE_EXCEPTION);
>         }{code}
> bookie thread is waiting at *this.join()* in below code
> {code}// Shutdown the ZK client
>                 if(zk != null) zk.close();
>                 // Shutdown journal
>                 journal.shutdown();
>                 this.join();
>                 syncThread.shutdown();
>                 // close Ledger Manager{code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira