You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-dev@hadoop.apache.org by "Hui Fei (Jira)" <ji...@apache.org> on 2022/08/15 11:50:00 UTC

[jira] [Reopened] (HDFS-16689) Standby NameNode crashes when transitioning to Active with in-progress tailer

     [ https://issues.apache.org/jira/browse/HDFS-16689?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hui Fei reopened HDFS-16689:
----------------------------

> Standby NameNode crashes when transitioning to Active with in-progress tailer
> -----------------------------------------------------------------------------
>
>                 Key: HDFS-16689
>                 URL: https://issues.apache.org/jira/browse/HDFS-16689
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: ZanderXu
>            Assignee: ZanderXu
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 3.4.0
>
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> Standby NameNode crashes when transitioning to Active with a in-progress tailer. And the error message like blew:
> {code:java}
> Caused by: java.lang.IllegalStateException: Cannot start writing at txid X when there is a stream available for read: ByteStringEditLog[X, Y], ByteStringEditLog[X, 0]
> 	at org.apache.hadoop.hdfs.server.namenode.FSEditLog.openForWrite(FSEditLog.java:344)
> 	at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.openForWrite(FSEditLogAsync.java:113)
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startActiveServices(FSNamesystem.java:1423)
> 	at org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.startActiveServices(NameNode.java:2132)
> 	... 36 more
> {code}
> After tracing and found there is a critical bug in *EditlogTailer#catchupDuringFailover()* when *DFS_HA_TAILEDITS_INPROGRESS_KEY* is true. Because *catchupDuringFailover()* try to replay all missed edits from JournalNodes with *onlyDurableTxns=true*. It may cannot replay any edits when they are some abnormal JournalNodes. 
> Reproduce method, suppose:
> - There are 2 namenode, namely NN0 and NN1, and the status of echo namenode is Active, Standby respectively. And there are 3 JournalNodes, namely JN0, JN1 and JN2. 
> - NN0 try to sync 3 edits to JNs with started txid 3, but only successfully synced them to JN1 and JN3. And JN0 is abnormal, such as GC, bad network or restarted.
> - NN1's lastAppliedTxId is 2, and at the moment, we are trying failover active from NN0 to NN1. 
> - NN1 only got two responses from JN0 and JN1 when it try to selecting inputStreams with *fromTxnId=3*  and *onlyDurableTxns=true*, and the count txid of response is 0, 3 respectively. JN2 is abnormal, such as GC,  bad network or restarted.
> - NN1 will cannot replay any Edits with *fromTxnId=3* from JournalNodes because the *maxAllowedTxns* is 0.
> So I think Standby NameNode should *catchupDuringFailover()* with *onlyDurableTxns=false* , so that it can replay all missed edits from JournalNode.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-dev-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-help@hadoop.apache.org