hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yang Jiandan (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (HDFS-12132) Both two NameNodes become Standby because the ZKFC exception
Date Fri, 14 Jul 2017 04:37:00 GMT

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

Yang Jiandan edited comment on HDFS-12132 at 7/14/17 4:36 AM:
--------------------------------------------------------------

Thank you for your response. I understand what you mean. ZKFC let NN2 also become standby,
and the whole HDFS is not available, which leads to Yarn and HBase are also not available.
so the consequences are too serious. The better solution is that the ZKFC throws exception
and exits, the most important is keeping the Active NameNode state unchanged.


was (Author: yangjiandan):
I understand what you mean. ZKFC let NN2 also become standby, and the whole HDFS is not available,
which leads to Yarn and HBase are also not available. so the consequences are too serious.
The better solution is that the ZKFC throws exception and exits, the most important is keeping
the Active NameNode state unchanged.

> Both two NameNodes become Standby because the ZKFC exception
> ------------------------------------------------------------
>
>                 Key: HDFS-12132
>                 URL: https://issues.apache.org/jira/browse/HDFS-12132
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: auto-failover
>    Affects Versions: 2.8.1
>            Reporter: Yang Jiandan
>
> Active NameNode become Standby because the ZKFC exception and Standby NameNode is still
Standby When rolling upgrading Hadoop from Hadoop-2.6.5 to Hadoop-2.8.0, this lead HDFS to
be not available. The logic of processing exception in ZKFC seems to be problematic, ZKFC
should guarantee to have a active NameNode.
> Before upgrading, the cluster was deployed with HA, NN1 was active, and NN2 was standby
> The configuration before upgrading is as follows:
> {code:java}
> dfs.namenode.rpc-address.nameservice.nn1 nn1: 8020
> dfs.namenode.rpc-address.nameservice.nn2 nn2: 8020
> {code}
> After upgrading, add the configuration of the separate RPC service:
> {code:java}
> dfs.namenode.rpc-address.nameservice.nn1 nn1: 8020
> dfs.namenode.rpc-address.nameservice.nn2 nn2: 8020
> dfs.namenode.servicerpc-address.nameservice.nn1 nn1: 8021
> dfs.namenode.servicerpc-address.nameservice.nn2 nn2: 8021
> dfs.namenode.lifeline.rpc-address.nameservice.nn1 nn1: 8022
> dfs.namenode.lifeline.rpc-address.nameservice.nn2 nn2: 8022
> {code}
> The upgrade steps are as follows:
> 1. Upgrade NN2: restart NameNode process on NN2
> 2. Upgrade NN1: restart the NameNode process on NN1, then NN2 becomes active, NN1 is
standby
> 3. Restart both ZKFC on NN1 and NN2
> After restarting ZKFC,  Active NameNodes have become Standby, and Standby NameNode did
not become Active. Two ZKFC having been doing a loop and threw many same exceptions.
> {code:java}
> createLockNodeAsync()  // create lock successfully
> becomeActive()  // return false
> terminateConnection()  // delete EPHEMERAL znode of 'ActiveStandbyElectorLock'  
> sleepFor(sleepTime)
> {code}
> After running command 'hdfs zkfc -formatZK', ZKFC backs to normal. 
> ZKFC Exception log is:
> {code:java}
> 2017-07-11 18:49:44,311 WARN [main-EventThread] org.apache.hadoop.ha.ActiveStandbyElector:
Exception handling the winning of election
> java.lang.RuntimeException: Mismatched address stored in ZK for NameNode at nn2/xx.xxx.xx.xxx:8022:
Stored protobuf was nameserviceId: “nameservice”
> namenodeId: "nn2"
> hostname: “nn2_hostname”
> port: 8020
> zkfcPort: 8019
> , address from our own configuration for this NameNode was nn2_hostname/xx.xxx.xx.xxx:8021
>         at org.apache.hadoop.hdfs.tools.DFSZKFailoverController.dataToTarget(DFSZKFailoverController.java:87)
>         at org.apache.hadoop.ha.ZKFailoverController.fenceOldActive(ZKFailoverController.java:506)
>         at org.apache.hadoop.ha.ZKFailoverController.access$1100(ZKFailoverController.java:61)
>         at org.apache.hadoop.ha.ZKFailoverController$ElectorCallbacks.fenceOldActive(ZKFailoverController.java:895)
>         at org.apache.hadoop.ha.ActiveStandbyElector.fenceOldActive(ActiveStandbyElector.java:985)
>         at org.apache.hadoop.ha.ActiveStandbyElector.becomeActive(ActiveStandbyElector.java:882)
>         at org.apache.hadoop.ha.ActiveStandbyElector.processResult(ActiveStandbyElector.java:467)
>         at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:599)
>         at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> 2017-07-11 18:49:44,311 INFO [main-EventThread] org.apache.hadoop.ha.ActiveStandbyElector:
Trying to re-establish ZK session
> 2017-07-11 18:49:44,311 INFO [main-EventThread] org.apache.zookeeper.ZooKeeper: Session:
0x15c3ada0ec319aa closed
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message