[
https://issues.apache.org/jira/browse/BOOKKEEPER-654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13715518#comment-13715518
]
Rakesh R commented on BOOKKEEPER-654:
-------------------------------------
bq.We should find why the operations are hanging and fix them, rather than just
adding checking
During bk#close(), it would shutdown the 'mainWorkerPool.shutdown();'. Now when
user tries to open a non-closed ledger with this bkclient, it would go for
recovery and mark ledgerInrecovery in zk metadata, then on zk callback its
submitting operations to the mainWorkerPool. Since the mainWorkerPool has
already closed, its throwing the following exception. In the otherside,
bookkeeper client call is infinitely waiting for the result, due to the
exception its not sending any results back.
{code}
2013-07-22 23:47:43,390 - ERROR - [main-EventThread:ClientCnxn$EventThread@623]
- Caught unexpected throwable
java.util.concurrent.RejectedExecutionException
at
java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:1774)
at
java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:768)
at
java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:656)
at
java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:78)
at
java.util.concurrent.Executors$DelegatedExecutorService.submit(Executors.java:599)
at
org.apache.bookkeeper.util.OrderedSafeExecutor.submitOrdered(OrderedSafeExecutor.java:92)
at
org.apache.bookkeeper.util.OrderedSafeExecutor$OrderedSafeGenericCallback.operationComplete(OrderedSafeExecutor.java:130)
at
org.apache.bookkeeper.meta.AbstractZkLedgerManager$3.processResult(AbstractZkLedgerManager.java:186)
at
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:545)
at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:497)
{code}
Here one observation is, bookkeeper client is not properly conveying the
message to the user saying 'BkClient is Closed and no operation permits',
secondly the user calls are allowed to change the ledger metadata through the
closed bookkeeper client.
> Bookkeeper client operations are allowed even after its closure, bk#close()
> ---------------------------------------------------------------------------
>
> Key: BOOKKEEPER-654
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-654
> Project: Bookkeeper
> Issue Type: Bug
> Components: bookkeeper-client
> Affects Versions: 4.2.0
> Reporter: Rakesh R
> Assignee: Rakesh R
> Fix For: 4.3.0
>
> Attachments: 0001-BOOKKEEPER-654-testcase-to-understand-more.patch,
> 0002-BOOKKEEPER-654.patch
>
>
> User can perform below operations with the closed bookkeeper client, which
> was instantiated with external zkclient.
> - open a closed ledger
> - create a new ledger
> Also, ledgerhandle operations like fencing/add/write are infinitely hanging.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira