[
https://issues.apache.org/jira/browse/BOOKKEEPER-244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13272091#comment-13272091
]
Rakesh R commented on BOOKKEEPER-244:
-------------------------------------
Oh, I could have attach the logs. Thanks for the analysis.
I have killed the Active NN after creating new ledger and inprogress znode is
in zookeeper(Before writing any entry in new ledger).
Following is the logs:
{noformat}
2012-05-09 16:55:11,349 INFO org.apache.zookeeper.ClientCnxn: Session
establishment complete on server 10.18.40.155/10.18.40.155:2181, sessionid =
0x137314c462c0009, negotiated timeout = 4000
2012-05-09 16:55:11,396 FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog:
Error: recoverUnfinalizedSegments failed for required journal
(JournalAndStream(mgr=org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager@193a83cc,
stream=null))
java.io.IOException: Exception retreiving last tx id for ledger [LedgerId:7,
firstTxId:18, lastTxId:-12345, version:-40]
at
org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.recoverLastTxId(BookKeeperJournalManager.java:516)
at
org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.recoverUnfinalizedSegments(BookKeeperJournalManager.java:418)
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)
Caused by: java.io.IOException: Error reading entries from bookkeeper
at
org.apache.hadoop.contrib.bkjournal.BookKeeperEditLogInputStream$LedgerInputStream.nextStream(BookKeeperEditLogInputStream.java:198)
at
org.apache.hadoop.contrib.bkjournal.BookKeeperEditLogInputStream$LedgerInputStream.read(BookKeeperEditLogInputStream.java:218)
at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
at java.io.FilterInputStream.read(FilterInputStream.java:66)
at
org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader$PositionTrackingInputStream.read(FSEditLogLoader.java:734)
at java.io.FilterInputStream.read(FilterInputStream.java:66)
at java.util.zip.CheckedInputStream.read(CheckedInputStream.java:42)
at java.io.DataInputStream.readByte(DataInputStream.java:248)
at
org.apache.hadoop.hdfs.server.namenode.FSEditLogOp$Reader.decodeOp(FSEditLogOp.java:2275)
at
org.apache.hadoop.hdfs.server.namenode.FSEditLogOp$Reader.readOp(FSEditLogOp.java:2248)
at
org.apache.hadoop.contrib.bkjournal.BookKeeperEditLogInputStream.nextOp(BookKeeperEditLogInputStream.java:100)
at
org.apache.hadoop.hdfs.server.namenode.EditLogInputStream.readOp(EditLogInputStream.java:74)
at
org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.recoverLastTxId(BookKeeperJournalManager.java:506)
... 22 more
Caused by: org.apache.bookkeeper.client.BKException$BKReadException
at org.apache.bookkeeper.client.BKException.create(BKException.java:48)
at
org.apache.bookkeeper.client.LedgerHandle.readEntries(LedgerHandle.java:302)
at
org.apache.hadoop.contrib.bkjournal.BookKeeperEditLogInputStream$LedgerInputStream.nextStream(BookKeeperEditLogInputStream.java:190)
... 35 more
2012-05-09 16:55:11,397 INFO org.apache.hadoop.contrib.bkjournal.WriteLock:
Zookeeper event WatchedEvent state:SyncConnected type:NodeDeleted
path:/ledgers/lock/lock-0000000005 received, reapplying watch to null
2012-05-09 16:55:11,400 INFO org.apache.hadoop.hdfs.server.namenode.NameNode:
SHUTDOWN_MSG:
/************************************************************
SHUTDOWN_MSG: Shutting down NameNode at HOST-10-18-40-91/10.18.40.91
************************************************************/
{noformat}
I feel in BKJM, no logic for a ledger which does not contain any entry.
I think if current ledger not contains any entry it should take end txnId from
previous ledger.
{noformat}
long endTxId = HdfsConstants.INVALID_TXID;
FSEditLogOp op = in.readOp();
while (op != null) {
if (endTxId == HdfsConstants.INVALID_TXID
|| op.getTransactionId() == endTxId+1) {
endTxId = op.getTransactionId();
}
op = in.readOp();
}
return endTxId;
{noformat}
> 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
>
> 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