[
https://issues.apache.org/jira/browse/HBASE-19753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16321416#comment-16321416
]
stack commented on HBASE-19753:
-------------------------------
Let me dump this info in here in case it helps...
I'm looking at flakies. It seems like TestHCM hangs on occasion because we
can't read clusterid using the ReadOnlyZKClient that is in the ZKAsyncRegistry.
The Master has written it (I have it dump the log when it thinks it wrote
it)... then later I see the ReadOnlyZKClient not being able to find it. The
Master startup fails because we are stuck at this spot. Here is a bit of log...
{code}
2018-01-10 15:43:06,512 INFO [M:0;ve0524:60789] master.HMaster(805): CLUSTERID
SET
2018-01-10 15:43:06,536 INFO [M:0;ve0524:60789]
balancer.BaseLoadBalancer(1037): slop=0.001, tablesOnMaster=false,
systemTablesOnMaster=false
2018-01-10 15:43:06,537 INFO [M:0;ve0524:60789]
balancer.StochasticLoadBalancer(214): Loaded config; maxSteps=1000000,
stepsPerRegion=800, maxRunningTime=30000, isByTable=false, etc.
2018-01-10 15:43:06,538 DEBUG [M:0;ve0524:60789] zookeeper.ZKUtil(355):
master:60789-0x160e26d00200010, quorum=localhost:50723, baseZNode=/hbase Set
watcher on existing znode=/hbase/balancer
2018-01-10 15:43:06,539 DEBUG [M:0;ve0524:60789] zookeeper.ZKUtil(357):
master:60789-0x160e26d00200010, quorum=localhost:50723, baseZNode=/hbase Set
watcher on znode that does not yet exist, /hbase/normalizer
2018-01-10 15:43:06,541 DEBUG [M:0;ve0524:60789] zookeeper.ZKUtil(357):
master:60789-0x160e26d00200010, quorum=localhost:50723, baseZNode=/hbase Set
watcher on znode that does not yet exist, /hbase/switch/split
2018-01-10 15:43:06,541 DEBUG [M:0;ve0524:60789] zookeeper.ZKUtil(357):
master:60789-0x160e26d00200010, quorum=localhost:50723, baseZNode=/hbase Set
watcher on znode that does not yet exist, /hbase/switch/merge
2018-01-10 15:43:06,543 DEBUG [Time-limited test-EventThread]
zookeeper.ZKWatcher(460): regionserver:35280-0x160e26d00200011,
quorum=localhost:50723, baseZNode=/hbase Received ZooKeeper Event,
type=NodeCreated, state=SyncConnected, path=/hbase/running
2018-01-10 15:43:06,543 DEBUG [Time-limited test-EventThread]
zookeeper.ZKWatcher(460): regionserver:57007-0x160e26d00200012,
quorum=localhost:50723, baseZNode=/hbase Received ZooKeeper Event,
type=NodeCreated, state=SyncConnected, path=/hbase/running
2018-01-10 15:43:06,543 DEBUG [Time-limited test-EventThread]
zookeeper.ZKWatcher(460): master:60789-0x160e26d00200010,
quorum=localhost:50723, baseZNode=/hbase Received ZooKeeper Event,
type=NodeCreated, state=SyncConnected, path=/hbase/running
2018-01-10 15:43:06,546 INFO [M:0;ve0524:60789]
zookeeper.ReadOnlyZKClient(129): Start read only zookeeper connection
0x655063fa to localhost:50723, session timeout 90000 ms, retries 30, retry
interval 1000 ms, keep alive 60000 ms
2018-01-10 15:43:18,316 INFO [ReadOnlyZKClient]
zookeeper.ReadOnlyZKClient(305): 0x1cbc2aa9 no activities for 60000 ms, close
active connection. Will reconnect next time when there are new requests.
2018-01-10 15:43:33,249 INFO [ReadOnlyZKClient]
zookeeper.ReadOnlyZKClient(305): 0x7dc94b7b 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}
I added threadumping if Master can't start and Master seems stuck here:
{code}
Thread 1646 (M:0;ve0524:60789):
State: WAITING
Blocked count: 74
Waited count: 130
Waiting on java.util.concurrent.CompletableFuture$Signaller@2b7d5aee
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:1035)
org.apache.hadoop.hbase.master.HMaster.initializeZKBasedSystemTrackers(HMaster.java:726)
org.apache.hadoop.hbase.master.HMaster.finishActiveMasterInitialization(HMaster.java:811)
org.apache.hadoop.hbase.master.HMaster.startActiveMasterManager(HMaster.java:2028)
org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:546)
java.lang.Thread.run(Thread.java:748)
{code}
The timeout for readonly is > how long we wait for Master to come up.
This latter seems to make for test flakeyness on my internal cluster at least.
Usually this TestHCM passes.
> Miscellany of fixes for hbase-zookeeper tests to make them more robust
> ----------------------------------------------------------------------
>
> Key: HBASE-19753
> URL: https://issues.apache.org/jira/browse/HBASE-19753
> Project: HBase
> Issue Type: Bug
> Reporter: stack
> Assignee: stack
> Attachments: HBASE-19753.branch-2.001.patch
>
>
> On my cluster which slows zk, tests hbase-zookeeper rarely all pass.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)