New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
recycle already. #1476
Comments
@infodog thank you for reporting this. will take a look at the stack trace and fix it. |
@infodog can you describe more about this issue?
|
@sijie my situation is when one process reading from the log, and one process writing to the log, the writer is fast. my code of reading is like following: while (true) {
|
@infodog - I think it is similar, it looks like a double-release issue. will take a closer look. |
@sijie - I found the problem, in bookkeeper-4.7.0/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java this function:
the request is issued twice one is So in org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal, the request is put in to the completionObjects twice with different txnid. so the request will be closed twice. We should use another request object to initiateSpeculativeRequest? The following is the logs showing the problem,we can see that cb.hashCode=126537557 is added twice with different txid. And the second one is added by 2018-06-09 17:17:56 [ BookKeeperClientWorker-OrderedExecutor-0-0:167553 ] - [ WARN ] txid->cb,txid=7534, cb.hashCode=126537557 org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal(PerChannelBookieClient.java:744) |
@infodog nice catch! I think the problem comes from speculative reads with object recycling, the problem exists both at speculative reads and speculative longpoll reads. the read objects are recycled without cancelling scheduled speculations. are you interested in providing a fix for that? |
@sijie OK, I interested in providing a fix. |
…ryOp Descriptions of the changes in this PR: The issue #1476 is caused by peculative reads with object recycling, same request object will be added to the CompletionObjects multiple times with different txnid. In fact the logic of process the request already take this into account, only on place inside `ReadLastConfirmedAndEntryOp.requestComplete` forget to check requestComplete before calling `submitCallback` which in turn call request.close. ### Motivation to fix #1476 ### Changes check `requestComplete` before `submitCallback` in `ReadLastConfirmedAndEntryOp.requestComplete` Master Issue: #1476 Author: Sijie Guo <sijie@apache.org> Author: infodog <infodog@hotmail.com> Author: zhengxiangyang <zxy@xinshi.net> Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Jia Zhai <None> This closes #1509 from infodog/issue1476, closes #1476 (cherry picked from commit 6476fc3) Signed-off-by: Sijie Guo <sijie@apache.org>
…AndEntryOp Descriptions of the changes in this PR: The issue apache#1476 is caused by peculative reads with object recycling, same request object will be added to the CompletionObjects multiple times with different txnid. In fact the logic of process the request already take this into account, only on place inside `ReadLastConfirmedAndEntryOp.requestComplete` forget to check requestComplete before calling `submitCallback` which in turn call request.close. ### Motivation to fix apache#1476 ### Changes check `requestComplete` before `submitCallback` in `ReadLastConfirmedAndEntryOp.requestComplete` Master Issue: apache#1476 Author: Sijie Guo <sijie@apache.org> Author: infodog <infodog@hotmail.com> Author: zhengxiangyang <zxy@xinshi.net> Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Jia Zhai <None> This closes apache#1509 from infodog/issue1476, closes apache#1476 (cherry picked from commit 6476fc3) Signed-off-by: Sijie Guo <sijie@apache.org> (cherry picked from commit c7b1610) Signed-off-by: JV Jujjuri <vjujjuri@salesforce.com>
bookkeeper 4.7, when using distributed log api, got the following error:
[BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalStateException: recycled already
at io.netty.util.Recycler$Stack.pushNow(Recycler.java:560)
at io.netty.util.Recycler$Stack.push(Recycler.java:550)
at io.netty.util.Recycler$DefaultHandle.recycle(Recycler.java:209)
at org.apache.bookkeeper.client.impl.LedgerEntryImpl.recycle(LedgerEntryImpl.java:163)
at org.apache.bookkeeper.client.impl.LedgerEntryImpl.close(LedgerEntryImpl.java:154)
at org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp$ReadLACAndEntryRequest.close(ReadLastConfirmedAndEntryOp.java:94)
at org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.submitCallback(ReadLastConfirmedAndEntryOp.java:530)
at org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.readEntryComplete(ReadLastConfirmedAndEntryOp.java:568)
at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1559)
at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1640)
at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV3Response(PerChannelBookieClient.java:1615)
at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1232)
at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
at java.lang.Thread.run(Thread.java:745)
The text was updated successfully, but these errors were encountered: