[
https://issues.apache.org/jira/browse/BOOKKEEPER-654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13750958#comment-13750958
]
Rakesh R commented on BOOKKEEPER-654:
-------------------------------------
bq.Actually, the real problem here is that we don't keep track of the
operations on a handle/client, and therefore cannot cancel them
I could see PerChanelBookieClient is holding the outstanding requests and I'm
just thinking to re-use to do cancelling?
{code}
ConcurrentHashMap<CompletionKey, AddCompletion> addCompletions = new
ConcurrentHashMap<CompletionKey, AddCompletion>();
ConcurrentHashMap<CompletionKey, ReadCompletion> readCompletions = new
ConcurrentHashMap<CompletionKey, ReadCompletion>();
{code}
bq.Rakesh's patch only resolves the case where the OrderedSafeExecutor has been
stopped before the request has been submitted. If the OrderedSafeExecutor is
stopped after the request has submitted, but the request also submits to the
executor, then the request callback will never get completed.
Good catch. I had missed this case.
bq.For example, if you have an add request. The request is submitted to the
executor fine, but then the executor is killed. The request goes to the server.
Once if comes back, PerChannelBookieClient#messageReceived will get a
RejectExecution when it tries to handle it, and your request is hung. I tried
to build on Rakesh's patch originally, but there's a hell of a lot of cases of
this, so I considered it cleaner to have a single point to cancelling callbacks.
I understood the idea of centralizing the callbacks using the wrappers. It
would be good if we could avoid the increase of callback chains(Sijie's comment
- 'extra referencing on callbacks'). I'll try today with the other idea of
cancel by using the PerChannelBookieClient's datastructures and wouldn't mind
choosing the better one. What are your thoughts on this idea?
> 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.2.2, 4.3.0
>
> Attachments:
> 0001-BOOKKEEPER-654-Bookkeeper-client-operations-are-allo.patch,
> 0001-BOOKKEEPER-654-testcase-to-understand-more.patch,
> 0002-BOOKKEEPER-654.patch, 0003-BOOKKEEPER-654.patch,
> 0004-BOOKKEEPER-654.patch, 0005-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