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

surendra singh lilhore commented on BOOKKEEPER-244:
---------------------------------------------------

In one more scenario BKJM fail to recover bad inProgress_ZNodes and throwing 
following Exception:

Say, NN has successfully done finalizeLogSegment() and unfortunately (kill the 
namenode or ZK cluster down) before deleting the inprogress  Znode. The 
inProgress_zNode would be left in the ZooKeeper. On next NN start up, it will 
again tries to perform recovery and throwing the following exception.
{noformat}
java.io.IOException: Node 
/NN/ledgers/edits_000000000000000039_000000000000000076 already exists but data 
doesn't match
        at 
org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.finalizeLogSegment(BookKeeperJournalManager.java:306)
        at 
org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.recoverUnfinalizedSegments(BookKeeperJournalManager.java:426)
        at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$6.apply(JournalSet.java:551)
        at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:322)
        at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.recoverUnfinalizedSegments(JournalSet.java:548)
        at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.recoverUnclosedStreams(FSEditLog.java:1134)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startActiveServices(FSNamesystem.java:598)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.startActiveServices(NameNode.java:1287)
        at 
org.apache.hadoop.hdfs.server.namenode.ha.ActiveState.enterState(ActiveState.java:61)
        at 
org.apache.hadoop.hdfs.server.namenode.ha.HAState.setStateInternal(HAState.java:63)
        at 
org.apache.hadoop.hdfs.server.namenode.ha.StandbyState.setState(StandbyState.java:49)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNode.transitionToActive(NameNode.java:1219)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.transitionToActive(NameNodeRpcServer.java:978)
        at 
org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB.transitionToActive(HAServiceProtocolServerSideTranslatorPB.java:107)
        at 
org.apache.hadoop.ha.proto.HAServiceProtocolProtos$HAServiceProtocolService$2.callBlockingMethod(HAServiceProtocolProtos.java:3633)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:427)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:916)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1692)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1688)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:396)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1232)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1686)
{noformat}

Here, I feel throw exception and will delete the old inprogress file because I 
have seen after restart the namenode /NamenodeNode/ledgers 
contain two inprogress znode.
                
> BookKeeperJournalManager: NN startup is failing, when tries to 
> recoverUnfinalizedSegments() a bad inProgress_ ZNodes
> --------------------------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-244
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-244
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Rakesh R
>            Assignee: Ivan Kelly
>
> Say, the InProgress_000X node is corrupted due to not writing the 
> data(version, ledgerId, firstTxId) to this inProgress_000X znode. Namenode 
> startup has the logic to recover all the unfinalized segments, here will try 
> to read the segment and getting shutdown.
> {noformat}
> EditLogLedgerMetadata.java:
> static EditLogLedgerMetadata read(ZooKeeper zkc, String path)
>       throws IOException, KeeperException.NoNodeException  {
>       byte[] data = zkc.getData(path, false, null);
>       String[] parts = new String(data).split(";");
>       if (parts.length == 3)
>          ....reading inprogress metadata
>       else if (parts.length == 4)
>          ....reading inprogress metadata
>       else
>         throw new IOException("Invalid ledger entry, "
>                               + new String(data));
>       }
> {noformat}
> Scenario:- Leaving bad inProgress_000X node ?
> Assume BKJM has created the inProgress_000X zNode and ZK is not available 
> when trying to add the metadata. Now, inProgress_000X ends up with partial 
> information.

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