[
https://issues.apache.org/jira/browse/HDFS-3423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277723#comment-13277723
]
Rakesh R commented on HDFS-3423:
--------------------------------
In the patch I'm seeing only maxTxId.reset(maxTxId.get()-1); is invoked on
SegmentEmptyException. But I'm thinking about the inprogress_x ledgers which
are not empty and had previously finalized but not deleted.
The following code I have taken from BKJM. When l.verify(zkc, finalisedPath) ==
true, here instead of storing the maxTxId and deleting the znode, we will only
delete the inprogress_x node as we had corresponding edit_x_y log file exists.
IMHO, this is more safer. what's your opinion?
{noformat}
try {
l.write(zkc, finalisedPath);
} catch (KeeperException.NodeExistsException nee) {
if (!l.verify(zkc, finalisedPath)) {
throw new IOException("Node " + finalisedPath + " already exists"
+ " but data doesn't match");
}
}
maxTxId.store(lastTxId);
zkc.delete(inprogressPath, inprogressStat.getVersion());
{noformat}
> BookKeeperJournalManager: NN startup is failing, when tries to
> recoverUnfinalizedSegments() a bad inProgress_ ZNodes
> --------------------------------------------------------------------------------------------------------------------
>
> Key: HDFS-3423
> URL: https://issues.apache.org/jira/browse/HDFS-3423
> Project: Hadoop HDFS
> Issue Type: Sub-task
> Reporter: Rakesh R
> Assignee: Ivan Kelly
> Attachments: HDFS-3423.diff
>
>
> 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