[
https://issues.apache.org/jira/browse/BOOKKEEPER-215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13252958#comment-13252958
]
Flavio Junqueira commented on BOOKKEEPER-215:
---------------------------------------------
This is the code for asyncReadEntries:
{noformat}
public void asyncReadEntries(long firstEntry, long lastEntry,
ReadCallback cb, Object ctx) {
// Little sanity check
if (firstEntry < 0 || lastEntry > lastAddConfirmed
|| firstEntry > lastEntry) {
cb.readComplete(BKException.Code.ReadException, this, null, ctx);
return;
}
try {
new PendingReadOp(this, firstEntry, lastEntry, cb, ctx).initiate();
} catch (InterruptedException e) {
cb.readComplete(BKException.Code.InterruptedException, this, null,
ctx);
}
}
{noformat}
initiate() is called form the application thread, right?
> 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