[
https://issues.apache.org/jira/browse/BOOKKEEPER-272?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440181#comment-13440181
]
Ivan Kelly commented on BOOKKEEPER-272:
---------------------------------------
{quote}
If I understand correctly, on NodeExistsException we are trying to append the
missingReplica to the underreplicated ledger so that will notifies about one
more down bookie which contains the ledger copy.
If we return simply, then the setData() method will not be called and there is
a chance of missing the info about second replica.
For Ex: L00001 ensemble BK1, BK2, BK3.
Say BK1 fails initially, then will markUnderreplicated ledger as L000001(BK1 as
the data).
Now again BK2 has failed, then while creating will get NEE, so we will append
BK2 also like: L000001(BK1 BK2).
I think "break; statement" is making sense and after that the duplicate entry
addition should be removed as per my latest patch.
Am I missing anything?
{quote}
The point of this code is, that if the node already exists, then there is
already a missing replica. We loop through the missingReplicas, to see if the
new missingReplica is already there or not. If so, then we can assume someone
else has reported this replica missing, so we return.
{quote}
Yeah I'll add main method also. But what about retaining start() and stop()
method as public. In future this will allow others(any external entity) to
manage the recovery process easily ?
{quote}
start() and stop() as public is fine. But if initialization fails, the ctor
should throw and exception. This way, the null checks are unneeded.
Regarding Service, we can look at that again after this is in. See my comment
on BOOKKEEPER-247.
> Provide automatic mechanism to know bookie failures
> ---------------------------------------------------
>
> Key: BOOKKEEPER-272
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-272
> Project: Bookkeeper
> Issue Type: Sub-task
> Components: bookkeeper-auto-recovery
> Reporter: Rakesh R
> Assignee: Rakesh R
> Fix For: 4.2.0
>
> Attachments: BOOKKEEPER-272.1.patch, BOOKKEEPER-272.2.patch,
> BOOKKEEPER-272.3.patch, BOOKKEEPER-272.Auditor.1.patch,
> BOOKKEEPER-272.Auditor.patch
>
>
> The idea is to build automatic mechanism to find out the bookie failures.
> Setup the bookie failure notifications to start the re-replication process.
> There are multiple approaches to findout bookie failures. Please refer the
> documents attached in BookKeeper-237.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira