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

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

I don't think we need to do that. from an application they call 
asyncReadEntries and asyncAddEntry, it has done the same thing to acquire lock. 
(one thing we might need to add is more documentation to let user know they 
acquire the lock when calling these two methods) 

Beside that, add operation is quite different from read operation.

for add operation, bookkeeper needs to guarantee order so it has a 
PendingAddOps queue to queue all the add operations, so the acquire is better 
to execute in LedgerHandle before putting it in the pending queue.

for read operation, we don't need to guarantee the order between different 
asycReadEntries calls, so we don't has a queue for read ops. beside that, read 
is actually a range read so we have to acquire the lock per entry for a 
asyncReadEntries operation, so PendingReadOp is a better place to manage such 
logic for a read op. 

does it make sense for you?

BTW, just curious, why the issue is marked as fixed?
                
> 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