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

Todd Lipcon commented on HDFS-3477:
-----------------------------------

Nice work. Thanks for fixing this. A few comments below:

- When you catch and swallow InterruptedException, you should call 
{{Thread.currentThread().interrupt()}} afterward, so that the interrupt status 
isn't lost

----
{code}
+  /**
+   * Watcher for waiting(connectionTimeout in milliseconds) to establish
+   * connection with the zookeeper server.
+   */
{code}
I don't think you need the parenthetical ("connectionTimeout in milliseconds") 
since you already specify that down below in the relevant javadoc.

----
- In the case that the ZK connection fails (eg ZK is just not listening) do we 
now have to wait for the full session timeout? I think the latch should fire on 
either SyncConnected or Disconnected events, not just on SyncConnected.
- We probably want some kind of sanity check that we don't get any other kind 
of events in the ZKConnectionWatcher. Otherwise I'm afraid we might end up 
swallowing an event inadvertently, which could result in the elector hanging. 
In that case, we should also make sure that the "authenticated" message we get 
from a SASL-capable ZK is in the list of acceptable events.

----
- Nit: can you make ZKConnectionWatcher private, and move it to the bottom of 
the file? Also, I think zkConnectLatch can be made private.
- Test nit: can you please use the 
{{GenericTestUtils.assertExceptionContains()}} utility to check the exception? 
I've found that the JUnit "expected exception" trick isn't great, since it 
catches all sorts of exceptions instead of just the targeted error you want to 
verify

                
> 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: 3.0.0
>            Reporter: suja s
>            Assignee: Rakesh R
>         Attachments: HDFS-3477.1.patch, HDFS-3477.patch
>
>
> 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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to