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

Rakesh R commented on BOOKKEEPER-610:
-------------------------------------

[~ikelly] oh, seems that I don't explain clearly at my previous comment.

Assume bookie is shutting down and invoked syncThread.shutdown(). Say 
ledgerStorage.flush(); throws NoWritableLedgerDirException, now this would
call dirsListener.allDisksFull();. Here it would do zk calls to 
Bookie#transitionToReadOnlyMode(), since zk is closed previosly, would get the 
following KeeperException. On KeeperException, we have logic to 
Bookie#triggerBookieshutdown() and this would again call Bookie.this.shutdown() 
on a new Thread and enters to deadlock situation.


{code}
org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = 
Session expired for 
        at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
        at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1021)
        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1049)
{code}

Bookie#transitionToReadOnlyMode() exception handling.
{code}
       } catch (KeeperException e) {
            LOG.error("Error in transition to ReadOnly Mode."
                    + " Shutting down", e);
            triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION);
            return;
        } 
{code}

{code}
    void triggerBookieShutdown(final int exitCode) {
        Thread shutdownThread = new Thread() {
            public void run() {
                Bookie.this.shutdown(exitCode);
            }
        };
        shutdownThread.start();
        try {
            shutdownThread.join();
        } catch (InterruptedException e) {
            Thread.currentThread().interrupt();
            LOG.debug("InterruptedException while waiting for shutdown. Not a 
problem!!");
        }
    }
{code}
                
> Make SyncThread use an executor
> -------------------------------
>
>                 Key: BOOKKEEPER-610
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-610
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Ivan Kelly
>            Assignee: Ivan Kelly
>             Fix For: 4.3.0
>
>         Attachments: 
> 0001-BOOKKEEPER-610-Make-SyncThread-use-an-executor.patch, 
> 0001-BOOKKEEPER-610-Make-SyncThread-use-an-executor.patch, 
> 0001-BOOKKEEPER-610-Make-SyncThread-use-an-executor.patch
>
>
> Currently we have a bunch of boolean variables to control the lifecycle of 
> the SyncThread. We're effectively replicating what an Executor does, so we 
> should just use an executor.

--
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

Reply via email to