You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by "Ivan Kelly (JIRA)" <ji...@apache.org> on 2012/06/08 16:31:24 UTC

[jira] [Commented] (BOOKKEEPER-272) Provide automatic mechanism to know bookie failures

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

Ivan Kelly commented on BOOKKEEPER-272:
---------------------------------------

New patch is a lot clearer. Comments follow;

- The changes to ServerConfiguration are unnesessary. The paths aren't actually configurable, so the shouldn't be in the configuration. Also, i don't think they should be configurable. I think that all auditor data should be under the auditor znode also. So eventually we'll have the three paths:
      /ledgers/auditor/election
      /ledgers/auditor/failedbookies
      /ledgers/auditor/underreplicated
      /ledgers/auditor/

- Election watcher does an election on the expired event. If you get this event, the bookie will shutdown (see Bookie.java)

- For the moment, and maybe for ever, whether the auditor runs should be configurable. During the course of 4.2 we're going to be building up this stuff incrementally. We should only enable it for tests which specifically use it (to start with).

- getMyInfo() should be called getIdentifier(). It would be great if you could modify registerBookie() to take a String rather than a port, so that the identifier for the bookie is only generated at one point ever.

- Collections.sort() on children isn't fully correct. V_1 & V_10 will sort before V_2, as it sorts alphabetically. Better to define a comparator, which splits on the '_', parses the number and does a comparison of that.

- Checking if you've won the election should be a matter of children.get(0).equals(myVote), once children is sorted correctly. 

- Check for specific exceptions. findbugs warns on catch (Exception e) because this also catches RuntimeExceptions. IOException is the wrong exception to use for this stuff also. Custom exceptions, defined under BookieException.

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

- I don't understand the point of writing auditor data to the auditor znode. It doesn't seem to do anything.

- The auditor code seems to check for failed bookies, and publishes these. We should skip a step here though, and instead of publishing the bookes, publish the ledgers which are on that bookie. In fact, it's much more efficient if it runs like this. If we only publish the failed bookies, each worker picking up a bookie to recover has to read all ledgers to find which ledgers to recover. If the auditor is maintaining a bookie -> ledger index, only one node needs to be reading all the ledgers.

- The run loop method of Auditor isn't a loop. I think the run loop should look as like:

{code}
public void run() {
   Set<String> bookies = getAvailableBookies();
   while (true) {
       waitForNotification();
       Set<String> newBookies = getAvailableBookies();
       Set<String> lostBookies = bookies;
       lostBookies.removeAll(newBookies);
       bookies = newBookies;

       if (lostBookies.size() > 0) {
           continue;
       }

       Map<String, List<Long>> bookie2ledgersMap = generateBookie2LedgersIndex();
       Set<Long> suspectedLedgers = HashSet<Long>();
       for (String b : lostBookies) {
           suspectedLedgers.addAll(bookie2ledgersMap.get(b));
       }
       publishSuspectedLedgers(suspectedLedgers);
   }
}
{code}

getAvailableBookies() should put a watch on the available znode, and fire a notification when triggered.
                
> 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