You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Bharath Vissapragada (Jira)" <ji...@apache.org> on 2020/03/30 20:14:00 UTC

[jira] [Commented] (HBASE-24075) [Flakey Tests] teardown fails because JmxCacheBuster wants to read ClusterId from closed fs

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

Bharath Vissapragada commented on HBASE-24075:
----------------------------------------------

Assigning this to myself.. something funky is happening with the FileSystem references between unit tests in a same test file. 

Our handling of FileSystem objects is super messy in the first place (we override hdfs shutdown handler to avoid races in hbase side teardown) but then it gets even messier in unit tests because the FileSystem object cache is long lived (static, hence per JVM instance) so they outlive a single unit test. My gut feeling is that something in the code is not properly cleaning up these instances, so the subsequent unit test in the same test file (which share the JVM) is using these bad references and throwing the error.

We can easily repro this by running a test file with more than 1 unit test. First unit test doesn't throw any exceptions but messes up the state before finishing and the subsequent test uses the bad FS reference and throws right away. I need to spend a little more time on this.

> [Flakey Tests] teardown fails because JmxCacheBuster wants to read ClusterId from closed fs
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-24075
>                 URL: https://issues.apache.org/jira/browse/HBASE-24075
>             Project: HBase
>          Issue Type: Bug
>          Components: flakies
>            Reporter: Michael Stack
>            Assignee: Bharath Vissapragada
>            Priority: Major
>
> I've seen this a few times. There may be an issue already but couldn't turn it up.
> In teardown, we fail because of below access:
> {code}
>  2020-03-28 22:01:47,484 WARN  [HBase-Metrics2-1] master.CachedClusterId(105): Error fetching cluster ID
>  java.io.IOException: Filesystem closed
>    at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:468)
>    at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1650)
>    at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1523)
>    at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1520)
>    at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>    at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1520)
>    at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1627)
>    at org.apache.hadoop.hbase.util.FSUtils.getClusterId(FSUtils.java:578)
>    at org.apache.hadoop.hbase.master.CachedClusterId.attemptFetch(CachedClusterId.java:103)
>    at org.apache.hadoop.hbase.master.CachedClusterId.getFromCacheOrFetch(CachedClusterId.java:140)
>    at org.apache.hadoop.hbase.master.HMaster.getClusterId(HMaster.java:3839)
>    at org.apache.hadoop.hbase.master.MetricsMasterWrapperImpl.getClusterId(MetricsMasterWrapperImpl.java:68)
>    at org.apache.hadoop.hbase.master.MetricsMasterSourceImpl.getMetrics(MetricsMasterSourceImpl.java:112)
>    at org.apache.hadoop.metrics2.impl.MetricsSourceAdapter.getMetrics(MetricsSourceAdapter.java:200)
>    at org.apache.hadoop.metrics2.impl.MetricsSourceAdapter.updateJmxCache(MetricsSourceAdapter.java:183)
>    at org.apache.hadoop.metrics2.impl.MetricsSourceAdapter.getMBeanInfo(MetricsSourceAdapter.java:156)
>    at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getNewMBeanClassName(DefaultMBeanServerInterceptor.java:333)
>    at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(DefaultMBeanServerInterceptor.java:319)
>    at com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(JmxMBeanServer.java:522)
>    at org.apache.hadoop.metrics2.util.MBeans.register(MBeans.java:66)
>    at org.apache.hadoop.metrics2.impl.MetricsSourceAdapter.startMBeans(MetricsSourceAdapter.java:223)
>    at org.apache.hadoop.metrics2.impl.MetricsSourceAdapter.start(MetricsSourceAdapter.java:101)
>    at org.apache.hadoop.metrics2.impl.MetricsSystemImpl.registerSource(MetricsSystemImpl.java:268)
>    at org.apache.hadoop.metrics2.impl.MetricsSystemImpl$1.postStart(MetricsSystemImpl.java:239)
>    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>    at java.lang.reflect.Method.invoke(Method.java:498)
>    at org.apache.hadoop.metrics2.impl.MetricsSystemImpl$3.invoke(MetricsSystemImpl.java:320)
>    at com.sun.proxy.$Proxy13.postStart(Unknown Source)
>    at org.apache.hadoop.metrics2.impl.MetricsSystemImpl.start(MetricsSystemImpl.java:193)
>    at org.apache.hadoop.metrics2.impl.JmxCacheBuster$JmxCacheBusterRunnable.run(JmxCacheBuster.java:109)
>    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>    at java.lang.Thread.run(Thread.java:748)
> {code}
> Its hadoop metrics system. Its still up trying to work.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)