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

Fei Hui commented on HDFS-14961:
--------------------------------

[~ayushtkn] Dig into the code. zkfc makes observer namenode election if health 
checking passed.
Maybe we should fix zkfc behavior, UT with sleep as you mentioned passed, code 
follows
{code}
  private void recheckElectability() {
    // Maintain lock ordering of elector -> ZKFC
    synchronized (elector) {
      synchronized (this) {
        boolean healthy = lastHealthState == State.SERVICE_HEALTHY;
    
        long remainingDelay = delayJoiningUntilNanotime - System.nanoTime(); 
        if (remainingDelay > 0) {
          if (healthy) {
            LOG.info("Would have joined master election, but this node is " +
                "prohibited from doing so for " +
                TimeUnit.NANOSECONDS.toMillis(remainingDelay) + " more ms");
          }
          scheduleRecheck(remainingDelay);
          return;
        }
    
        switch (lastHealthState) {
        case SERVICE_HEALTHY:
          if(serviceState != HAServiceState.OBSERVER) {
            elector.joinElection(targetToData(localTarget));
          }
          if (quitElectionOnBadState) {
            quitElectionOnBadState = false;
          }
          break;
{code}

code changes are
{quote}
          if(serviceState != HAServiceState.OBSERVER) {
            elector.joinElection(targetToData(localTarget));
          }
{quote}
 

> 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-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: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to