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

Sijie Guo commented on BOOKKEEPER-400:
--------------------------------------

@Aniruddha, ah, yes. these two functions would be called from one thread. but 
the analysis of the steps are still right, since we don't block 
sendAddSuccessCallbacks when handleBookieFailure in the old code.

so handleBookieFailure would first sending changeEnsemble request to persist 
new ensemble to zookeeper. before its callback, the entries before (and 
include) 2935 are proceeding #sendAddSuccessCallbacks to remove them from 
pendingAddOps queue. after changeEnsemble callbacked, it doesn't unset those 
removed pendingAddOps.

@Ivan, blockAddCompletions in BOOKKEEPER-208 doesn't fix this issue here, since 
it was placed in a wrong place. it would be better to place it before sending 
changeEnsemble request not in its callback. Let me explain as below:

1) pendingAddOps has requests for (1, 2, 3)
2) 3 failed to added a bookie, enter handleBookieFailure to change ensemble 
starting from 1.
3) handleBookieFailure send change ensemble request to ZooKeeper. 
blockAddCompletions is still zero.
4) 1 callbacked to sendAddSuccessCallbacks, which remove 1 from pendingAddOps, 
since blockAddCompletions is zero.
5) 2 callbacked to sendAddSuccessCallbacks, which remove 2 from pendingAddOps, 
since blockAddCompletions is zero 2.
6) changeEnsemble callbacked. blockAddCompletions is incremented to 1 now. but 
it doesn't take any effects. since 1 and 2 have been already removed from 
pendingAddOps. so the ensemble is changed without reissue 1 and 2 requests to 
the changed bookie, which caused the problem here.

a straightforward fix is to increment blockAddCompletions before sending 
changing ensemble request.

besides that, I found that unsetSuccessAndSendWriteRequest is now not follow 
the guideline that all operations for a ledger should be submitted to same 
thread, which is introduced when fixing BOOKKEEPER-337. (thanks Aniruddha's 
reminder)

unsetSuccessAndSendWriteRequest would be called in ReReadLedgerMetadataCb 
directly, which is run in zookeeper's thread not bookkeeper's worker thread. it 
would be better to protect ReReadLedgerMetadataCb is submitted back to 
bookkeeper's worker thread as what we did for ChangeEnsembleCb.
                
> Ledger entry not found in any of the bookies in the ensemble responsible for 
> that entry.
> ----------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-400
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-400
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>            Reporter: Aniruddha
>         Attachments: clean.log.gz
>
>
> Detailed discussion at 
> http://mail-archives.apache.org/mod_mbox/zookeeper-bookkeeper-dev/201209.mbox/%3cCAOLhyDQzrmeOHmTxzPikeAqJ7pZUn0=vHfd=gc1srmtuye5...@mail.gmail.com%3e
> We had an internal discussion about this. From BOOKKEEPER-337, it seems that 
> handleBookieFailure could be invoked in parallel by a thread other the one 
> that calls LedgerHandle#sendAddSuccessCallbacks. The values updated by 
> handleBookieFailure might not be visible to the thread running 
> sendAddSuccessCallbacks because the fields are not volatile and this might 
> have caused our bad state. 
> BK-337 synchronizes access to metadata.addEnsemble() and we believe this 
> would make this scenario very improbable. 
> A long term fix might be to make LedgerMetadata immutable since it is rarely 
> updated. 

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