[ https://issues.apache.org/jira/browse/HBASE-27957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Viraj Jasani updated HBASE-27957: --------------------------------- Component/s: Client > HConnection (and ZookeeprWatcher threads) leak in case of AUTH_FAILED > exception. > -------------------------------------------------------------------------------- > > Key: HBASE-27957 > URL: https://issues.apache.org/jira/browse/HBASE-27957 > Project: HBase > Issue Type: Bug > Components: Client > Affects Versions: 1.7.2, 2.4.17 > Reporter: Rushabh Shah > Priority: Critical > > Observed this in production environment running some version of 1.7 release. > Application didn't had the right keytab setup for authentication. Application > was trying to create HConnection and zookeeper server threw AUTH_FAILED > exception. > After few hours of application in this state, saw thousands of > zk-event-processor thread with below stack trace. > {noformat} > "zk-event-processor-pool1-t1" #1275 daemon prio=5 os_prio=0 cpu=1.04ms > elapsed=41794.58s tid=0x00007fd7805066d0 nid=0x1245 waiting on condition > [0x00007fd75df01000] > java.lang.Thread.State: WAITING (parking) > at jdk.internal.misc.Unsafe.park(java.base@11.0.18.0.102/Native > Method) > - parking to wait for <0x00007fd9874a85e0> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > at > java.util.concurrent.locks.LockSupport.park(java.base@11.0.18.0.102/LockSupport.java:194) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(java.base@11.0.18.0.102/AbstractQueuedSynchronizer.java:2081) > at > java.util.concurrent.LinkedBlockingQueue.take(java.base@11.0.18.0.102/LinkedBlockingQueue.java:433) > at > java.util.concurrent.ThreadPoolExecutor.getTask(java.base@11.0.18.0.102/ThreadPoolExecutor.java:1054) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.18.0.102/ThreadPoolExecutor.java:1114) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.18.0.102/ThreadPoolExecutor.java:628) > {noformat} > {code:java|title=ConnectionManager.java|borderStyle=solid} > HConnectionImplementation(Configuration conf, boolean managed, > ExecutorService pool, User user, String clusterId) throws IOException > { > ... > ... > try { > this.registry = setupRegistry(); > retrieveClusterId(); > ... > ... > } catch (Throwable e) { > // avoid leaks: registry, rpcClient, ... > LOG.debug("connection construction failed", e); > close(); > throw e; > } > {code} > retrieveClusterId internally calls ZKConnectionRegistry#getClusterId > {code:java|title=ZKConnectionRegistry.java|borderStyle=solid} > private String clusterId = null; > @Override > public String getClusterId() { > if (this.clusterId != null) return this.clusterId; > // No synchronized here, worse case we will retrieve it twice, that's > // not an issue. > try (ZooKeeperKeepAliveConnection zkw = > hci.getKeepAliveZooKeeperWatcher()) { > this.clusterId = ZKClusterId.readClusterIdZNode(zkw); > if (this.clusterId == null) { > LOG.info("ClusterId read in ZooKeeper is null"); > } > } catch (KeeperException | IOException e) { ---> WE ARE SWALLOWING > THIS EXCEPTION AND RETURNING NULL. > LOG.warn("Can't retrieve clusterId from Zookeeper", e); > } > return this.clusterId; > } > {code} > ZkConnectionRegistry#getClusterId threw the following exception.(Our logging > system trims stack traces longer than 5 lines.) > {noformat} > Cause: org.apache.zookeeper.KeeperException$AuthFailedException: > KeeperErrorCode = AuthFailed for /hbase/hbaseid > StackTrace: > org.apache.zookeeper.KeeperException.create(KeeperException.java:126) > org.apache.zookeeper.KeeperException.create(KeeperException.java:54) > org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1213) > org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:285) > org.apache.hadoop.hbase.zookeeper.ZKUtil.checkExists(ZKUtil.java:470) > {noformat} > We should throw KeeperException from ZKConnectionRegistry#getClusterId all > the way back to HConnectionImplementation constructor to close all the > watcher threads and throw the exception back to the caller. -- This message was sent by Atlassian Jira (v8.20.10#820010)