You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Nick Dimiduk (Jira)" <ji...@apache.org> on 2020/02/05 18:54:00 UTC
[jira] [Created] (HBASE-23801) ExecutorService ignores Throwables
Nick Dimiduk created HBASE-23801:
------------------------------------
Summary: ExecutorService ignores Throwables
Key: HBASE-23801
URL: https://issues.apache.org/jira/browse/HBASE-23801
Project: HBase
Issue Type: Bug
Components: regionserver
Affects Versions: 2.3.0
Reporter: Nick Dimiduk
From the [test logs|https://builds.apache.org/job/HBase%20Nightly/job/branch-2/2451/testReport/junit/org.apache.hadoop.hbase.mapred/TestTableMapReduce/health_checks___yetus_jdk8_hadoop2_checks___/] of a failed build, I find that the {{ExecutorService}} running in the region server appears to ignore exceptions thrown in its executor's tasks. In this case, a fatal {{RuntimeException}} was thrown. Instead of aborting the RS, we merely log. This results in the region open operation never completing, and the test simply hangs.
This implementation is ancient, dating back to HBASE-4281 or before, circa 2011. Thus I assume it impacts all active release lines.
{noformat}
2020-02-05 07:45:53,361 FATAL [RS_OPEN_REGION-regionserver/e590165b0ef0:0-1] conf.Configuration(2859): error parsing conf hbase-site.xml
org.xml.sax.SAXParseException; systemId: file:/home/jenkins/jenkins-slave/workspace/HBase_Nightly_branch-2@2/component/hbase-mapreduce/target/test-classes/hbase-site.xml; lineNumber: 1; columnNumber: 1; Premature end of file.
at com.sun.org.apache.xerces.internal.parsers.DOMParser.parse(DOMParser.java:257)
at com.sun.org.apache.xerces.internal.jaxp.DocumentBuilderImpl.parse(DocumentBuilderImpl.java:339)
at javax.xml.parsers.DocumentBuilder.parse(DocumentBuilder.java:150)
at org.apache.hadoop.conf.Configuration.parse(Configuration.java:2684)
at org.apache.hadoop.conf.Configuration.parse(Configuration.java:2672)
at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2746)
at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2706)
at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2579)
at org.apache.hadoop.conf.Configuration.get(Configuration.java:1091)
at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1145)
at org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:1550)
at org.apache.hadoop.hbase.HBaseConfiguration.checkDefaultsVersion(HBaseConfiguration.java:70)
at org.apache.hadoop.hbase.HBaseConfiguration.addHbaseResources(HBaseConfiguration.java:84)
at org.apache.hadoop.hbase.HBaseConfiguration.create(HBaseConfiguration.java:98)
at org.apache.hadoop.hbase.util.LossyCounting.<init>(LossyCounting.java:84)
at org.apache.hadoop.hbase.regionserver.MetricsUserAggregateImpl.<init>(MetricsUserAggregateImpl.java:45)
at org.apache.hadoop.hbase.regionserver.MetricsUserAggregateFactory.getMetricsUserAggregate(MetricsUserAggregateFactory.java:35)
at org.apache.hadoop.hbase.regionserver.MetricsRegion.<init>(MetricsRegion.java:40)
at org.apache.hadoop.hbase.regionserver.HRegion.<init>(HRegion.java:849)
at org.apache.hadoop.hbase.regionserver.HRegion.<init>(HRegion.java:730)
at sun.reflect.GeneratedConstructorAccessor23.newInstance(Unknown Source)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.hadoop.hbase.regionserver.HRegion.newHRegion(HRegion.java:7135)
at org.apache.hadoop.hbase.regionserver.HRegion.openHRegionFromTableDir(HRegion.java:7352)
at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7329)
at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7287)
at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7238)
at org.apache.hadoop.hbase.regionserver.handler.AssignRegionHandler.process(AssignRegionHandler.java:133)
at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:104)
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)
{noformat}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)