hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Todd Lipcon (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (HDFS-3477) FormatZK and ZKFC startup can fail due to zkclient connection establishment delay
Date Sat, 10 Nov 2012 01:37:13 GMT

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

Todd Lipcon updated HDFS-3477:
------------------------------

    Attachment: hdfs-3477.txt

Hey Rakesh,

I modified your patch a bit and came up with this new rev. I'm afraid there might have been
a race in the old version -- if multiple events came in to the registration watcher, it might
have missed forwarding the second to the main watcher, which could produce an inconsistent
state. I changed it around a bit so that the existing watcher wrapper takes care of the latching.

If you're able to reproduce this in a real cluster, can you give it a try? I've never seen
the problem in practice, but my patch passes the unit test you added.
                
> FormatZK and ZKFC startup can fail due to zkclient connection establishment delay
> ---------------------------------------------------------------------------------
>
>                 Key: HDFS-3477
>                 URL: https://issues.apache.org/jira/browse/HDFS-3477
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>          Components: auto-failover
>    Affects Versions: 2.0.0-alpha
>            Reporter: suja s
>            Assignee: Rakesh R
>         Attachments: HDFS-3477.1.patch, HDFS-3477.2.patch, HDFS-3477.3.patch, HDFS-3477.3.patch,
HDFS-3477.patch, hdfs-3477.txt
>
>
> Format and ZKFC startup flows continue further after creation of zkclient connection
without waiting to check whether the connection is completely established. This  leads to
failure at the subsequent point if connection was not complete by then.
> Exception trace for format 
> {noformat}
> 12/05/30 19:48:24 INFO zookeeper.ClientCnxn: Socket connection established to HOST-xx-xx-xx-55/xx.xx.xx.55:2182,
initiating session
> 12/05/30 19:48:24 INFO zookeeper.ClientCnxn: Session establishment complete on server
HOST-xx-xx-xx-55/xx.xx.xx.55:2182, sessionid = 0x1379da4660c0014, negotiated timeout = 5000
> 12/05/30 19:48:24 WARN ha.ActiveStandbyElector: Ignoring stale result from old client
with sessionId 0x1379da4660c0014
> 12/05/30 19:48:24 INFO zookeeper.ZooKeeper: Session: 0x1379da4660c0014 closed
> 12/05/30 19:48:24 INFO zookeeper.ClientCnxn: EventThread shut down
> Exception in thread "main" java.io.IOException: Couldn't determine existence of znode
'/hadoop-ha/hacluster'
>         at org.apache.hadoop.ha.ActiveStandbyElector.parentZNodeExists(ActiveStandbyElector.java:263)
>         at org.apache.hadoop.ha.ZKFailoverController.formatZK(ZKFailoverController.java:257)
>         at org.apache.hadoop.ha.ZKFailoverController.doRun(ZKFailoverController.java:195)
>         at org.apache.hadoop.ha.ZKFailoverController.access$000(ZKFailoverController.java:58)
>         at org.apache.hadoop.ha.ZKFailoverController$1.run(ZKFailoverController.java:163)
>         at org.apache.hadoop.ha.ZKFailoverController$1.run(ZKFailoverController.java:159)
>         at org.apache.hadoop.security.SecurityUtil.doAsLoginUserOrFatal(SecurityUtil.java:438)
>         at org.apache.hadoop.ha.ZKFailoverController.run(ZKFailoverController.java:159)
>         at org.apache.hadoop.hdfs.tools.DFSZKFailoverController.main(DFSZKFailoverController.java:171)
> Caused by: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode
= ConnectionLoss for /hadoop-ha/hacluster
>         at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>         at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1021)
>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1049)
>         at org.apache.hadoop.ha.ActiveStandbyElector.parentZNodeExists(ActiveStandbyElector.java:261)
>         ... 8 more
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message