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

Sijie Guo commented on BOOKKEEPER-215:
--------------------------------------

I agreed to separated it into three jiras and we fix the first one.

> For adds, we could acquire the permits before doAsyncAddEntry, while for the 
> release part, we may need a different callback in PendingAddOp.

for Add it is OK, but how about readEntries? since releasing permits in 
PendingReadOp is not in single place, if you want to fix it in a different 
callback, I think you have to wrap the callback passed by user and do release 
in the wrapper. 

What I am thinking to resolve recovery deadlock issue is to revisit the 
throttle mechanism used in bookkeeper.
                
> 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
>            Assignee: Sijie Guo
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BK-215-check-deadlock.patch, BK-215.patch, 
> BK-215.patch_v2, BK-215.patch_v3, DeadlockCheckOrderedSafeExecutor.java, 
> 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