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

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

thanks Flavio for reviewing.

> but not the one in doRecoveryRead. Why do we need to give it to a worker 
> thread in this case?

doRecoveryRead() will be executed in #addComplete, which is also in the 
callback of bookkeeper. so it should do same as what it did in #readComplete.

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

if we use '+' to concatenate string it would introduce the performance issue. 
but we don't concatenate the string in debug message, so it would not introduce 
the performance issue. besides that, I remembered that because we use slf4j, 
Ivan suggested to use such kind of pattern to log debug message.

> Is this change gratuitous or really necessary:

actually I used this method in hedwig BookKeeperTestBase class, which extends 
existed Bookie class to simulate response delay for #readEntry. It helps 
testing the deadlock in bookkeeper persistence manager.

> testRecoveryDeadlockWithLimitedPermits() has no assertion or fail clause. 
> What is it testing?

if we don't apply the patch, the test case will hang due to deadlock. so I am 
not sure how to add assertion and fail clause. what is your opinion?

> I'm not entirely sure why we need this method:

the method is used in TestDeadLock#getServerConfiguration. it used to load a 
bookkeeper client configuration object. so the hub server could use bookkeeper 
client settings we provided.

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

yes. we have to wait until timeout if we don't apply this patch. I have no 
perfect solution to test such hang due to deadlock.

> typos & comments

yeah. thanks for fixing them. I would update them to new patch.


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