[ https://issues.apache.org/jira/browse/HBASE-19694?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16321690#comment-16321690 ]
Duo Zhang commented on HBASE-19694: ----------------------------------- For TestHCM, I think the problem is the timeout test... We should split them out. Let me have a try. > 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 > > > 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)