hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-19694) The initialization order for a fresh cluster is incorrect
Date Thu, 11 Jan 2018 09:43:00 GMT

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

stack commented on HBASE-19694:
-------------------------------

Testing on my wonky cluster, TestHCM is failing because Master gets stuck trying to read clusterid
with its ROZKClient. I see the ROZKClient stuck here:

Thread 1632 (M:0;ve0524:42980):
  State: WAITING
  Blocked count: 64
  Waited count: 120
  Waiting on java.util.concurrent.CompletableFuture$Signaller@1692dc09
  Stack:                                                                                 
                                                                                         
                                                                     sun.misc.Unsafe.park(Native
Method)
    java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
    java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
    java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)               
                                                                                         
                                                                     java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
    java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)              
                                                                                         
                                                                     org.apache.hadoop.hbase.client.ConnectionImplementation.retrieveClusterId(ConnectionImplementation.java:526)
    org.apache.hadoop.hbase.client.ConnectionImplementation.<init>(ConnectionImplementation.java:286)
    org.apache.hadoop.hbase.client.ConnectionUtils$ShortCircuitingClusterConnection.<init>(ConnectionUtils.java:141)
                                                                                         
                                          org.apache.hadoop.hbase.client.ConnectionUtils$ShortCircuitingClusterConnection.<init>(ConnectionUtils.java:132)
                                                                                         
                                          org.apache.hadoop.hbase.client.ConnectionUtils.createShortCircuitConnection(ConnectionUtils.java:185)
                                                                                         
                                                     org.apache.hadoop.hbase.regionserver.HRegionServer.createClusterConnection(HRegionServer.java:770)
    org.apache.hadoop.hbase.regionserver.HRegionServer.setupClusterConnection(HRegionServer.java:801)
                                                                                         
                                                         org.apache.hadoop.hbase.master.HMaster.createServerManager(HMaster.java:1032)
                                                                                         
                                                                             org.apache.hadoop.hbase.master.HMaster.finishActiveMasterInitialization(HMaster.java:803)
    org.apache.hadoop.hbase.master.HMaster.startActiveMasterManager(HMaster.java:2025)
    org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:546)                         
                                                                                         
                                                                     java.lang.Thread.run(Thread.java:748)

We get stuck here till test times out.

Just before the above we logged this... Its after the master sets the cluster id up into zk.

{code}
2018-01-11 01:24:51,243 INFO  [M:0;ve0524:42980] zookeeper.ReadOnlyZKClient(130): Start read
only zookeeper connection 0x054e47d4 to localhost:54728, session timeout 90000 ms, retries
30, retry interval 1000 ms, keep alive 60000 ms
...
2018-01-11 01:25:01,246 WARN  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(296): Failed connecting
after waiting 10000ms; State:CONNECTING sessionid:0x0 local:null remoteserver:null lastZxid:0
xid:1 sent:0 recv:0 queuedpkts:0 pendingresp:0 queuedevents:0
....
2018-01-11 01:25:05,500 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(314): 0x187ca22c
no activities for 60000 ms, close active connection. Will reconnect next time when there are
new requests.
2018-01-11 01:25:18,761 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(314): 0x02ab1506
no activities for 60000 ms, close active connection. Will reconnect next time when there are
new requests.
Process Thread Dump: Thread dump because Master not active after 30000 seconds
...
{code}

Above has us waiting on session connection for 10seconds but it don't happen.

> The initialization order for a fresh cluster is incorrect
> ---------------------------------------------------------
>
>                 Key: HBASE-19694
>                 URL: https://issues.apache.org/jira/browse/HBASE-19694
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Duo Zhang
>            Assignee: stack
>            Priority: Critical
>             Fix For: 2.0.0-beta-1
>
>         Attachments: HBASE-19694.branch-2.001.patch, HBASE-19694.branch-2.002.patch,
HBASE-19694.branch-2.003.patch, HBASE-19694.branch-2.004.patch, HBASE-19694.branch-2.005.patch,
HBASE-19694.branch-2.006.patch, HBASE-19694.branch-2.007.patch, HBASE-19694.branch-2.008.patch,
HBASE-19694.branch-2.009.patch
>
>
> The cluster id will set once we become the active master in finishActiveMasterInitialization,
but the blockUntilBecomingActiveMaster and finishActiveMasterInitialization are both called
in a thread to make the constructor of HMaster return without blocking. And since HMaster
itself is also a HRegionServer, it will create a Connection and then start calling reportForDuty.
And when creating the ConnectionImplementation, we will read the cluster id from zk, but the
cluster id may have not been set yet since it is set in another thread, we will get an exception
and use the default cluster id instead.
> I always get this when running UTs which will start a mini cluster
> {noformat}
> 2018-01-03 15:16:37,916 WARN  [M:0;zhangduo-ubuntu:32848] client.ConnectionImplementation(528):
Retrieve cluster id failed
> java.util.concurrent.ExecutionException: org.apache.zookeeper.KeeperException$NoNodeException:
KeeperErrorCode = NoNode for /hbase/hbaseid
> 	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
> 	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
> 	at org.apache.hadoop.hbase.client.ConnectionImplementation.retrieveClusterId(ConnectionImplementation.java:526)
> 	at org.apache.hadoop.hbase.client.ConnectionImplementation.<init>(ConnectionImplementation.java:286)
> 	at org.apache.hadoop.hbase.client.ConnectionUtils$ShortCircuitingClusterConnection.<init>(ConnectionUtils.java:141)
> 	at org.apache.hadoop.hbase.client.ConnectionUtils$ShortCircuitingClusterConnection.<init>(ConnectionUtils.java:137)
> 	at org.apache.hadoop.hbase.client.ConnectionUtils.createShortCircuitConnection(ConnectionUtils.java:185)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.createClusterConnection(HRegionServer.java:781)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.setupClusterConnection(HRegionServer.java:812)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.preRegistrationInitialization(HRegionServer.java:827)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:938)
> 	at org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:550)
> 	at java.lang.Thread.run(Thread.java:748)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
for /hbase/hbaseid
> 	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
> 	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> 	at org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient$ZKTask$1.exec(ReadOnlyZKClient.java:163)
> 	at org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient.run(ReadOnlyZKClient.java:311)
> 	... 1 more
> {noformat}



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

Mime
View raw message