hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Rakesh R (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-3441) Race condition between rolling logs at active NN and purging at standby
Date Fri, 18 May 2012 06:53:16 GMT

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

Rakesh R commented on HDFS-3441:
--------------------------------

Yeah, I have seen a race condition between the purgeLogsOlderThan() by Standby and the finalizeLogSegment()
by Active.

Cause: Following are the sequence of operations happening:
1) When standby comes to purge, it is reading all the list of ledger logs including the inprogress_72
files. 
2) Meantime Active NN is finalizing the logSegment inprogress_72 and creating new inprogress_74.

3) Now the Standby is reading the data of inprogress_72 to decide whether its inprogress or
not and is throwing NoNodeException.


I feel, the filtering of inprogress file could be done based on the file name itself, instead
of reading the content and filtering based on the data like as follows:

BookKeeperJournalManager.java
{noformat}
    List<String> ledgerNames = zkc.getChildren(ledgerPath, false);
      for (String ledgerName : ledgerNames) {
        if( !inProgressOk && ledgerName.contains("inprogress") ){
          continue;
        }
        ledgers.add(EditLogLedgerMetadata.read(zkc, ledgerPath + "/" + ledgerName));
      }
    } catch (Exception e) {
      throw new IOException("Exception reading ledger list from zk", e);
    }
{noformat}

                
> Race condition between rolling logs at active NN and purging at standby
> -----------------------------------------------------------------------
>
>                 Key: HDFS-3441
>                 URL: https://issues.apache.org/jira/browse/HDFS-3441
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>            Reporter: suja s
>
> Standby NN has got the ledgerlist with list of all files, including the inprogress file
(with say inprogress_val1)
> Active NN has done finalization and created new inprogress file.
> Standby when proceeds further finds that the inprogress file which it had in the list
is not present and NN gets shutdown
> NN Logs
> =========
> 2012-05-17 22:15:03,867 INFO org.apache.hadoop.hdfs.server.namenode.FSImage: Image file
of size 201 saved in 0 seconds.
> 2012-05-17 22:15:03,874 INFO org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer:
Triggering log roll on remote NameNode /xx.xx.xx.102:8020
> 2012-05-17 22:15:03,923 INFO org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager:
Going to retain 2 images with txid >= 111
> 2012-05-17 22:15:03,923 INFO org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager:
Purging old image FSImageFile(file=/home/May8/hadoop-3.0.0-SNAPSHOT/hadoop-root/dfs/name/current/fsimage_0000000000000000109,
cpktTxId=0000000000000000109)
> 2012-05-17 22:15:03,961 FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error:
purgeLogsOlderThan 0 failed for required journal (JournalAndStream(mgr=org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager@142e6767,
stream=null))
> java.io.IOException: Exception reading ledger list from zk
> 	at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.getLedgerList(BookKeeperJournalManager.java:531)
> 	at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.purgeLogsOlderThan(BookKeeperJournalManager.java:444)
> 	at org.apache.hadoop.hdfs.server.namenode.JournalSet$5.apply(JournalSet.java:541)
> 	at org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:322)
> 	at org.apache.hadoop.hdfs.server.namenode.JournalSet.purgeLogsOlderThan(JournalSet.java:538)
> 	at org.apache.hadoop.hdfs.server.namenode.FSEditLog.purgeLogsOlderThan(FSEditLog.java:1011)
> 	at org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.purgeOldStorage(NNStorageRetentionManager.java:98)
> 	at org.apache.hadoop.hdfs.server.namenode.FSImage.purgeOldStorage(FSImage.java:900)
> 	at org.apache.hadoop.hdfs.server.namenode.FSImage.saveFSImageInAllDirs(FSImage.java:885)
> 	at org.apache.hadoop.hdfs.server.namenode.FSImage.saveNamespace(FSImage.java:822)
> 	at org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer.doCheckpoint(StandbyCheckpointer.java:157)
> 	at org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer.access$900(StandbyCheckpointer.java:52)
> 	at org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer$CheckpointerThread.doWork(StandbyCheckpointer.java:279)
> 	at org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer$CheckpointerThread.access$300(StandbyCheckpointer.java:200)
> 	at org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer$CheckpointerThread$1.run(StandbyCheckpointer.java:220)
> 	at org.apache.hadoop.security.SecurityUtil.doAsLoginUserOrFatal(SecurityUtil.java:512)
> 	at org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer$CheckpointerThread.run(StandbyCheckpointer.java:216)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
for /nnedits/ledgers/inprogress_72
> 	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
> 	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> 	at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1113)
> 	at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1142)
> 	at org.apache.hadoop.contrib.bkjournal.EditLogLedgerMetadata.read(EditLogLedgerMetadata.java:113)
> 	at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.getLedgerList(BookKeeperJournalManager.java:528)
> 	... 16 more
> 2012-05-17 22:15:03,963 INFO org.apache.hadoop.hdfs.server.namenode.NameNode: SHUTDOWN_MSG:

> ZK Data
> ================
> [zk: xx.xx.xx.55:2182(CONNECTED) 9] get /nnedits/ledgers/inprogress_74
> -40;59;116
> cZxid = 0x2be
> ctime = Thu May 17 22:15:03 IST 2012
> mZxid = 0x2be
> mtime = Thu May 17 22:15:03 IST 2012
> pZxid = 0x2be
> cversion = 0
> dataVersion = 0
> aclVersion = 0
> ephemeralOwner = 0x0
> dataLength = 10
> numChildren = 0

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