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

stack commented on HBASE-12554:
-------------------------------

Looks like the commit adding debugging and timeouts surfaced the root cause of 
zombie TestBaseLoadBalancer.

The code you have added here has the balancer wait a minute and then just moves 
on.  The 60seconds should be configurable.  Does the cancel actually interrupt 
the ongoing lookup or does it leave it hanging?  The implementation of 
DNSToSwitchMapping is primitive.  We could do better (use more of 
CachedDNSToSwitchMapping).  I like the idea of complaint if this lookup is 
going on too long.  Could punt all this to a new issue.

To fix the breaking test, RackManager loads an instance of class specified at  
hbase.util.ip.to.rack.determiner which is an implementation of 
DNSToSwitchMapping  Why not in the test put in a mock of DNSToSwitchMapping 
that returns immediately.  Who cares about a lookup in test? Why wait?



> TestBaseLoadBalancer may timeout due to lengthy rack lookup
> -----------------------------------------------------------
>
>                 Key: HBASE-12554
>                 URL: https://issues.apache.org/jira/browse/HBASE-12554
>             Project: HBase
>          Issue Type: Test
>            Reporter: Ted Yu
>            Assignee: Ted Yu
>         Attachments: 12554-v1.txt
>
>
> Here is one of the recent occurrences 
> (https://builds.apache.org/job/PreCommit-HBASE-Build/11778/console):
> {code}
> testImmediateAssignment(org.apache.hadoop.hbase.master.balancer.TestBaseLoadBalancer)
>   Time elapsed: 30.019 sec  <<< ERROR!
> java.lang.Exception: test timed out after 30000 milliseconds
>       at java.net.Inet4AddressImpl.lookupAllHostAddr(Native Method)
>       at java.net.InetAddress$1.lookupAllHostAddr(InetAddress.java:901)
>       at 
> java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1293)
>       at java.net.InetAddress.getAllByName0(InetAddress.java:1246)
>       at java.net.InetAddress.getAllByName(InetAddress.java:1162)
>       at java.net.InetAddress.getAllByName(InetAddress.java:1098)
>       at java.net.InetAddress.getByName(InetAddress.java:1048)
>       at org.apache.hadoop.net.NetUtils.normalizeHostName(NetUtils.java:561)
>       at org.apache.hadoop.net.NetUtils.normalizeHostNames(NetUtils.java:578)
>       at 
> org.apache.hadoop.net.CachedDNSToSwitchMapping.resolve(CachedDNSToSwitchMapping.java:109)
>       at 
> org.apache.hadoop.hbase.master.RackManager.getRack(RackManager.java:66)
>       at 
> org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer$Cluster.<init>(BaseLoadBalancer.java:273)
>       at 
> org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.createCluster(BaseLoadBalancer.java:1113)
>       at 
> org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.randomAssignment(BaseLoadBalancer.java:1175)
>       at 
> org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.immediateAssignment(BaseLoadBalancer.java:1145)
>       at 
> org.apache.hadoop.hbase.master.balancer.TestBaseLoadBalancer.testImmediateAssignment(TestBaseLoadBalancer.java:136)
> {code}
> One possible fix is to submit CachedDNSToSwitchMapping.resolve() to executor 
> pool for execution. RackManager.getRack() can set a timeout beyond which 
> UNKNOWN_RACK is returned.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to