hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ivan Kelly (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-3464) BKJM: Deleting currentLedger and leaving 'inprogress_x' on exceptions can throw BKNoSuchLedgerExistsException later.
Date Fri, 15 Jun 2012 14:42:42 GMT

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

Ivan Kelly commented on HDFS-3464:
----------------------------------

Have you seen this exception with the latest code? From what I see it's not possible.

{code}
    try {
      String znodePath = inprogressZNode(txId);
      EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
          HdfsConstants.LAYOUT_VERSION,  currentLedger.getId(), txId);
      /* Write the ledger metadata out to the inprogress ledger znode
       * This can fail if for some reason our write lock has
       * expired (@see WriteLock) and another process has managed to
       * create the inprogress znode.
       * In this case, throw an exception. We don't want to continue
       * as this would lead to a split brain situation.
       */
      l.write(zkc, znodePath);

      maxTxId.store(txId);
      ci.update(znodePath);
      return new BookKeeperEditLogOutputStream(conf, currentLedger);
    } catch (KeeperException ke) {
      cleanupLedger(currentLedger);
      throw new IOException("Error storing ledger metadata", ke);
    }
{code}
Only l.write() can throw an KeeperException, and in this case, it can only be a NodeExistsException
(we should perhaps tighten the catch). In the case of NodeExists, we do want to delete the
ledger, but not the inprogress znode. Otherwise, only a IOException will be thrown, and the
BKNoSuchLedgerExistsException shouldn't happen.
                
> BKJM: Deleting currentLedger and leaving 'inprogress_x'  on exceptions can throw BKNoSuchLedgerExistsException
later.
> ---------------------------------------------------------------------------------------------------------------------
>
>                 Key: HDFS-3464
>                 URL: https://issues.apache.org/jira/browse/HDFS-3464
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>    Affects Versions: 2.0.1-alpha, 3.0.0
>            Reporter: Uma Maheswara Rao G
>            Assignee: Uma Maheswara Rao G
>
> HDFS-3058 will clean currentLedgers on exception.
> In BookKeeperJournalManager, startLogSegment() is deleting the corresponding 'inprogress_ledger'
ledger on exception. Here leaving the 'inprogress_x' ledger metadata in ZooKeeper. When the
other node becomes active, he will see the 'inprogress_x' znode and tries to recoverLastTxId()
it would throw exception, since there is no 'inprogress_ledger' exists. 
> {noformat}
> Caused by: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException
> 	at org.apache.bookkeeper.client.BookKeeper.openLedger(BookKeeper.java:393)
> 	at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.recoverLastTxId(BookKeeperJournalManager.java:493)
> {noformat}
> As per the discussion in HDFS-3058, we will handle the coment as part of this JIRA.

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

        

Mime
View raw message