[ https://issues.apache.org/jira/browse/HBASE-21535?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16723288#comment-16723288 ]
Pankaj Kumar commented on HBASE-21535: -------------------------------------- My bad, sorry I missed somehow in the master branch patch. Attached V2, Kindly review. Thank you :) > Zombie Master detector is not working > ------------------------------------- > > Key: HBASE-21535 > URL: https://issues.apache.org/jira/browse/HBASE-21535 > Project: HBase > Issue Type: Bug > Components: master > Affects Versions: 3.0.0, 2.2.0, 2.1.1, 2.0.3 > Reporter: Pankaj Kumar > Assignee: Pankaj Kumar > Priority: Critical > Fix For: 3.0.0, 2.2.0 > > Attachments: HBASE-21535.branch-2.patch, HBASE-21535.branch-2.patch, > HBASE-21535.patch, HBASE-21535.v2.patch > > > We have InitializationMonitor thread in HMaster which detects Zombie Hmaster > based on _hbase.master.initializationmonitor.timeout _and halts if > _hbase.master.initializationmonitor.haltontimeout_ set _true_. > After HBASE-19694, HMaster initialization order was correted. Hmaster is set > active after Initializing ZK system trackers as follows, > {noformat} > status.setStatus("Initializing ZK system trackers"); > initializeZKBasedSystemTrackers(); > status.setStatus("Loading last flushed sequence id of regions"); > try { > this.serverManager.loadLastFlushedSequenceIds(); > } catch (IOException e) { > LOG.debug("Failed to load last flushed sequence id of regions" > + " from file system", e); > } > // Set ourselves as active Master now our claim has succeeded up in zk. > this.activeMaster = true; > {noformat} > But Zombie detector thread is started at the begining phase of > finishActiveMasterInitialization(), > {noformat} > private void finishActiveMasterInitialization(MonitoredTask status) throws > IOException, > InterruptedException, KeeperException, ReplicationException { > Thread zombieDetector = new Thread(new InitializationMonitor(this), > "ActiveMasterInitializationMonitor-" + System.currentTimeMillis()); > zombieDetector.setDaemon(true); > zombieDetector.start(); > {noformat} > During zombieDetector execution "master.isActiveMaster()" will be false, so > it won't wait and cant detect zombie master. > {noformat} > @Override > public void run() { > try { > while (!master.isStopped() && master.isActiveMaster()) { > Thread.sleep(timeout); > if (master.isInitialized()) { > LOG.debug("Initialization completed within allotted tolerance. Monitor > exiting."); > } else { > LOG.error("Master failed to complete initialization after " + timeout + "ms. > Please" > + " consider submitting a bug report including a thread dump of this > process."); > if (haltOnTimeout) { > LOG.error("Zombie Master exiting. Thread dump to stdout"); > Threads.printThreadInfo(System.out, "Zombie HMaster"); > System.exit(-1); > } > } > } > } catch (InterruptedException ie) { > LOG.trace("InitMonitor thread interrupted. Existing."); > } > } > } > {noformat} -- This message was sent by Atlassian JIRA (v7.6.3#76005)