[
https://issues.apache.org/jira/browse/BOOKKEEPER-400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13454680#comment-13454680
]
Rakesh R commented on BOOKKEEPER-400:
-------------------------------------
Sijie, Aniruddha, you both have done good analysis and brings out more
concurrent scenarios on write failures.
@Sijie @Aniruddha
bq.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)
BTW, this has not been introduced as part of BOOKKEEPER-337 and was there
before also. But honestly saying, while refactoring I haven't noticed the
zookeeper thread usage and I could have corrected it as part of that.
Could you please look at the following code snippet, which was there before
BOOKKEEPER-337 changes:
{code}
rereadMetadata(new GenericCallback<LedgerMetadata>() {
@Override
public void operationComplete(int newrc, LedgerMetadata newMeta) {
if (newrc != BKException.Code.OK) {
LOG.error("Error reading new metadata from ledger after changing
ensemble, code=" + newrc);
handleUnrecoverableErrorDuringAdd(rc);
} else {
// a new ensemble is added only when the start entry is larger than
zero
if (newEnsembleStartEntry > 0) {
metadata.getEnsembles().remove(newEnsembleStartEntry);
}
if (metadata.resolveConflict(newMeta)) {
metadata.addEnsemble(newEnsembleStartEntry, newEnsemble);
writeLedgerConfig(new ChangeEnsembleCb());
return;
} else {
LOG.error("Could not resolve ledger metadata conflict while
changing ensemble to: "
+ newEnsemble + ", old meta data is \n" + new
String(metadata.serialize())
+ "\n, new meta data is \n" + new
String(newMeta.serialize()) + "\n ,closing ledger");
handleUnrecoverableErrorDuringAdd(rc);
}
}
}
});
{code}
-Rakesh
> 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