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

Rakesh R commented on HDFS-3058:
--------------------------------

Hi Ivan,

Its great work and good to see the hot standby flows. Just few thoughts over 
the patch.

# 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}
# Also, can we do maxTxId.store(txId); only when finalizeLogSegment(). I'm 
thinking to limit standby do tailing only from 'edit_logs' ?

-Rakesh
                
> HA: Bring BookKeeperJournalManager up to date with HA changes
> -------------------------------------------------------------
>
>                 Key: HDFS-3058
>                 URL: https://issues.apache.org/jira/browse/HDFS-3058
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>            Reporter: Ivan Kelly
>            Assignee: Ivan Kelly
>             Fix For: 0.24.0
>
>         Attachments: HDFS-3058.diff, HDFS-3058.diff, HDFS-3058.diff
>
>
> There's a couple of TODO(HA) comments in the BookKeeperJournalManager code. 
> This JIRA is to address those.

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

        

Reply via email to