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

Ivan Kelly commented on BOOKKEEPER-509:
---------------------------------------

Just to clarify, the root cause here is that PendingReadOp is completing twice. 
I think the solution is good but could be simpler. In instead of having 
"isNewCompletedRequest", you could just have:
{code}
if (entry.complete(rctx.to, buffer)) {
    numPendingEntries--;
    
    if (numPendingEntries == 0) {
        submitCallback(BKException.Code.OK);
    }
} 
{code}

                
> TestBookKeeperPersistenceManager failed on latest trunk
> -------------------------------------------------------
>
>                 Key: BOOKKEEPER-509
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-509
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: hedwig-server
>            Reporter: Sijie Guo
>            Assignee: Sijie Guo
>            Priority: Blocker
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-509.diff, BOOKKEEPER-509.diff, 
> hedwig-server.log, hedwig-server.log.bak
>
>
> latest trunk failed at TestBookKeeperPersistenceManager.
> sees that it caused by uncaught exception:
> {code}
> java.util.NoSuchElementException
>         at java.util.AbstractQueue.remove(AbstractQueue.java:90)
>         at 
> org.apache.bookkeeper.client.PendingReadOp.nextElement(PendingReadOp.java:345)
>         at 
> org.apache.bookkeeper.client.PendingReadOp.nextElement(PendingReadOp.java:53)
>         at 
> org.apache.bookkeeper.client.LedgerRecoveryOp.readComplete(LedgerRecoveryOp.java:100)
>         at 
> org.apache.bookkeeper.client.PendingReadOp.submitCallback(PendingReadOp.java:338)
>         at 
> org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:327)
>         at 
> org.apache.bookkeeper.proto.PerChannelBookieClient.handleReadResponse(PerChannelBookieClient.java:627)
>         at 
> org.apache.bookkeeper.proto.PerChannelBookieClient$7.safeRun(PerChannelBookieClient.java:529)
>         at org.apache.bookkeeper.util.SafeRunnable.run(SafeRunnable.java:31)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:680)
> {code}

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

Reply via email to