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

Flavio Junqueira commented on BOOKKEEPER-215:
---------------------------------------------

Hi Sijie, Here are some comments on the patch:

* One clarification. I understand the call to 
lh.bk.callbackWorker.submitOrdered in readComplete, but not the one in 
doRecoveryRead. Why do we need to give it to a worker thread in this case?
* It is not performance critical in this case, but it sounds like a good ideal 
in general to have LOG.debug statements wrapped with isDebugEnabled() 
(LedgerRecoveryOp:86). You may have simply missed this one.
* Is this change gratuitous or really necessary:
{noformat}
    protected Bookie newBookie(ServerConfiguration conf)
        throws IOException, KeeperException, InterruptedException, 
BookieException {
        return new Bookie(conf);
    }

{noformat} 

* testRecoveryDeadlockWithLimitedPermits() has no assertion or fail clause. 
What is it testing?
* I'm not entirely sure why we need this method:

{noformat}
    /**
     * Add configuration object.
     *
     * @param conf configuration object
     */
    public void addConf(Configuration otherConf) throws ConfigurationException {
        conf.addConfiguration(otherConf);
    }
{noformat}
Why can't we set the bk client configuration in the constructor?

* Typo: "... so a scan request need to scan over two ledger" -> "... so a scan 
request need to scan over two ledgers"
* In TestDeadlock, if I understand the test correctly, consumeQueue.take() is 
supposed to hang due to the bug of this jira. Consequently, we have to wait 
until junit times out the test? I was wondering if there is a way of avoiding 
the time out.
* Suggestion for rephrasing comment:

{noformat}
// it obtains the permit and wait for a response,
// but the response is delayed and readEntries is called
// in the readComplete callback to read entries of the 
// same ledger. since there is no permit, it blocks.
{noformat}

                
> Deadlock occurs under high load
> -------------------------------
>
>                 Key: BOOKKEEPER-215
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-215
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: hedwig-server
>    Affects Versions: 4.1.0
>            Reporter: Aniruddha
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BK-215.patch, hedwig_ts.log
>
>
> LedgerHandle uses a Semaphore(opCounterSem) with a default value of 5000 
> permits to implement throttling for outstanding requests. This is causing a 
> deadlock under high load. What I've observed is the following - There are a 
> fixed number of threads created by OrderedSafeExecutor(mainWorkerPool in 
> BookKeeper) and this is used to execute operations by PerChannelBookieClient. 
> Under high load, the bookies are not able to satisfy requests at the rate at 
> which they are being generated. This exhausts all permits in the Semaphore 
> and any further operations block on lh.opCounterSem.acquire(). In this 
> scenario, if the connection to the bookies is shut down, channelDisconnected 
> in PerChannelBookieClient tries to error out all outstanding entries. The 
> errorOutReadKey and errorOutAddKey functions enqueue these operations in the 
> same mainWorkerPool, all threads in which are blocked on acquire. So, 
> handleBookieFailure is never executed and the server stops responding. 
> Blocking operations in a fixed size thread pool doesn't sound quite right. 
> Temporarily, I fixed this by having another ExecutorService for every 
> PerChannelBookieClient and queuing the operations from the errorOut* 
> functions in it, but this is just a quick fix. I feel that the server 
> shouldn't rely on LedgerHandle to throttle connections, but do this itself. 
> Any other ideas on how to fix this? I'd be happy to contribute a patch. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to