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

Chao Sun commented on HDFS-14961:
---------------------------------

Thanks [~ayushtkn]. The analysis and the fix also looks good to me. The race 
conditions is between the manual failover and the ZKFC auto failover. There is 
a gap between when the NN state is changed and when that is reflected on the ZK 
failover controller, through the health monitor, which can contribute to this.

In the log I also observed this though:

{code}
2019-11-22 11:04:41,630 [IPC Server handler 7 on default port 10021] INFO  
namenode.FSImage (FSEditLogLoader.java:loadFSEdits(178)) - Starbility(797)) - 
Would have joined master election, but this node is prohibited from doing so 
for 9896 more ms
...
2019-11-22 11:04:41,733 [IPC Server handler 2 on default port 
10024-EventThread] INFO  ha.ZKFailoverController 
(ZKFailoverController.java:becomeStandby(491)) - ZK Election indicated that 
NameNode at localhost/127.0.0.1:10022 should become standby
{code}

It is interesting that the SBN, after ceding active role, is supposed to rejoin 
the election after ~10s. But it happened much sooner than that and therefore 
caused the race condition.

One minor comment on the doc:
{code}
    // This is to eliminate any race condition between manually transition of
    // namenode into Observer, when the namenode has already participated in the
    // ZKFC election, before transition to Observer state as Standby Node.
    // For more details check : HDFS-14961.
{code}
Maybe we should change it to
{code}
    // This is to eliminate any race condition between manually transition of
    // namenode into Observer, and ZKFC auto failover election, when the 
namenode has already participated in the
    // ZKFC election, before transition to Observer state as Standby Node.
    // For more details check : HDFS-14961.
{code}





> [SBN read] Prevent ZKFC changing Observer Namenode state
> --------------------------------------------------------
>
>                 Key: HDFS-14961
>                 URL: https://issues.apache.org/jira/browse/HDFS-14961
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: Íñigo Goiri
>            Assignee: Ayush Saxena
>            Priority: Major
>         Attachments: HDFS-14961-01.patch, HDFS-14961-02.patch, 
> HDFS-14961-03.patch, ZKFC-TEST-14961.patch
>
>
> HDFS-14130 made ZKFC aware of the Observer Namenode and hence allows ZKFC 
> running along with the observer NOde.
> The Observer namenode isn't suppose to be part of ZKFC election process.
> But if the  Namenode was part of election, before turning into Observer by 
> transitionToObserver Command. The ZKFC still sends instruction to the 
> Namenode as a result of previous participation and sometimes tend to change 
> the state of Observer to Standby.
> This is also the reason for  failure in TestDFSZKFailoverController.
> TestDFSZKFailoverController has been consistently failing with a time out 
> waiting in testManualFailoverWithDFSHAAdmin(). In particular 
> {{waitForHAState(1, HAServiceState.OBSERVER);}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to