[
https://issues.apache.org/jira/browse/BOOKKEEPER-272?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13292802#comment-13292802
]
Rakesh R commented on BOOKKEEPER-272:
-------------------------------------
Thanks Ivan, again for the detailed comments. I'd like to know more on the
following:
{quote}
Instead of watching a specific node, watch for the ChildrenChanged event on the
zkAuditorElectPath. Even the current winner of the election can watch this.
When triggered, doElection should be fine, though you should shutdown the
Auditor thread if running if you loose the election.
{quote}
I have used the predecessor watching approach, used to avoid the herd effect
with zookeeper leader election. Rule is, bookie will be watching to my
predecessor bookie based on the ephemeral seq id. At any point of time, least
ephemeral znode bookie only will get the chance to become Auditor. So I thought
it would be more efficient to watch on my predecessor bookie. How does it sound?
{quote}
I don't understand the point of writing auditor data to the auditor znode. It
doesn't seem to do anything.
{quote}
Hope you meant, after the auditor election I'm keeping the auditor's myInfo to
the auditor election path(/ledgers/auditor/election). At present there is no
logic, I've just kept for debugging purpose only(using ZK znodes).
Below one:
{noformat}
auditorElector.zkclient.setData(auditorElector.conf
+ .getZkAuditorElectionPath(), auditorElector.myInfo
+ .getBytes(), -1);
{noformat}
{quote}
The run loop method of Auditor isn't a loop. I think the run loop should look
as like:
{quote}
Yup, I'll remove the volatile flag and do other modifications.
Here I'd like to keep the generate & publishing suspected ledgers before
entering to the waitForNotification(). Consider the scenario where the auditor
bookie(say BK1) has failed and new auditor(BK2) comes, he will not recieve any
notifications for the BK1 failures(from the available bookie's path).
> 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-server
> Reporter: Rakesh R
> Assignee: Rakesh R
> Attachments: BOOKKEEPER-272.1.patch, BOOKKEEPER-272.2.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