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

Aniruddha edited comment on BOOKKEEPER-215 at 4/11/12 10:38 PM:
----------------------------------------------------------------

Hi Ivan and Flavio, 

I've attached the thread dump of the hedwig-server when it is deadlocked. As 
you can see, both pool-1-thread-1 and pool-52-thread-1 are blocked on the 
semaphore. pool-52-thread-1 is one of the threads in the mainWorkerPool. It 
seems that PendingReadOp is executed in pool-1-thread-1 always, except for just 
before I get the error, when it's executed in pool-52-thread-1. Now, when the 
channel is disconnected, the errorOut* functions enqueue all their 
SafeRunnables in this same thread (using OrderedSafeExecutor.submitOrdered with 
ledgerId as the key) - pool-52-thread-1. Even when PerChannelBookieClient 
closes the channel after errorOutOutstandingEntries, the callbacks that would 
release the semaphore are never called because all of them are being queued in 
the same thread. So, now you have a PendingReadOp waiting to acquire a lock at 
the front of the queue while operations that will release the lock are queued 
behind it. 

In short, there is a PendingReadOp being queued in pool-52-thread-1 which 
shouldn't be there. The only operations in that thread should be PendingAddOps. 

edit : From the logfile it seems that PendingReadOps are being queued in 
pool-52-thread-1 quite a few times. In all previous instances, it could acquire 
the Semaphore. 
                
      was (Author: i0exception):
    Hi Ivan and Flavio, 

I've attached the thread dump of the hedwig-server when it is deadlocked. As 
you can see, both pool-1-thread-1 and pool-52-thread-1 are blocked on the 
semaphore. pool-52-thread-1 is one of the threads in the mainWorkerPool. It 
seems that PendingReadOp is executed in pool-1-thread-1 always, except for just 
before I get the error, when it's executed in pool-52-thread-1. Now, when the 
channel is disconnected, the errorOut* functions enqueue all their 
SafeRunnables in this same thread (using OrderedSafeExecutor.submitOrdered with 
ledgerId as the key) - pool-52-thread-1. Even when PerChannelBookieClient 
closes the channel after errorOutOutstandingEntries, the callbacks that would 
release the semaphore are never called because all of them are being queued in 
the same thread. So, now you have a PendingReadOp waiting to acquire a lock at 
the front of the queue while operations that will release the lock are queued 
behind it. 

In short, there is a PendingReadOp being queued in pool-52-thread-1 which 
shouldn't be there. The only operations in that thread should be PendingAddOps. 
                  
> 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
>         Attachments: 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