Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
2.4.8, 3.0.3, 3.1.2, 3.2.1
-
None
-
None
Description
This is another Spark deadlock caused by setting Hadoop FsUrlStreamHandlerFactory, similar to SPARK-31922 and SPARK-26961. We use Spark 2.4.8 and Hadoop 2.7.3 with a few cherry picks.
"spark-listener-group-shared": waiting to lock monitor 0x0000563048a4c7b8 (object 0x0000000680d8b560, a org.apache.hadoop.conf.Configuration), which is held by "main" "main": waiting to lock monitor 0x00007fd919ac26f8 (object 0x00000006802b6d68, a java.lang.Object), which is held by "spark-listener-group-shared"
"main": at java.net.URL.getURLStreamHandler(URL.java:1242) waiting to lock <0x00000006802b6d68> (a java.lang.Object) at java.net.URL.<init>(URL.java:617) at java.net.URL.<init>(URL.java:508) at java.net.URL.<init>(URL.java:457) at java.net.JarURLConnection.parseSpecs(JarURLConnection.java:175) at java.net.JarURLConnection.<init>(JarURLConnection.java:158) at sun.net.www.protocol.jar.JarURLConnection.<init>(JarURLConnection.java:81) at sun.net.www.protocol.jar.Handler.openConnection(Handler.java:41) at java.net.URL.openConnection(URL.java:1027) at java.net.URL.openStream(URL.java:1093) at java.util.ServiceLoader.parse(ServiceLoader.java:304) at java.util.ServiceLoader.access$200(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.hasNextService(ServiceLoader.java:357) at java.util.ServiceLoader$LazyIterator.hasNext(ServiceLoader.java:393) at java.util.ServiceLoader$1.hasNext(ServiceLoader.java:474) at javax.xml.parsers.FactoryFinder$1.run(FactoryFinder.java:293) at java.security.AccessController.doPrivileged(Native Method) at javax.xml.parsers.FactoryFinder.findServiceProvider(FactoryFinder.java:289) at javax.xml.parsers.FactoryFinder.find(FactoryFinder.java:267) at javax.xml.parsers.DocumentBuilderFactory.newInstance(DocumentBuilderFactory.java:119) at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2563) at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2539) at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2452) locked <0x0000000680d8b560> (a org.apache.hadoop.conf.Configuration) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1028) at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1078) at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2236) at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2654) at org.apache.hadoop.fs.FsUrlStreamHandlerFactory.createURLStreamHandler(FsUrlStreamHandlerFactory.java:74) at java.net.URL.getURLStreamHandler(URL.java:1190) at java.net.URL.<init>(URL.java:617) at java.net.URL.<init>(URL.java:508) at java.net.URL.<init>(URL.java:457) at java.net.JarURLConnection.parseSpecs(JarURLConnection.java:175) at java.net.JarURLConnection.<init>(JarURLConnection.java:158) at sun.net.www.protocol.jar.JarURLConnection.<init>(JarURLConnection.java:81) at sun.net.www.protocol.jar.Handler.openConnection(Handler.java:41) at java.net.URL.openConnection(URL.java:1027) at java.net.URL.openStream(URL.java:1093) at java.util.ServiceLoader.parse(ServiceLoader.java:304) at java.util.ServiceLoader.access$200(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.hasNextService(ServiceLoader.java:357) at java.util.ServiceLoader$LazyIterator.hasNext(ServiceLoader.java:393) at java.util.ServiceLoader$1.hasNext(ServiceLoader.java:474) at javax.xml.parsers.FactoryFinder$1.run(FactoryFinder.java:293) at java.security.AccessController.doPrivileged(Native Method) at javax.xml.parsers.FactoryFinder.findServiceProvider(FactoryFinder.java:289) at javax.xml.parsers.FactoryFinder.find(FactoryFinder.java:267) at javax.xml.parsers.DocumentBuilderFactory.newInstance(DocumentBuilderFactory.java:119) at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2563) at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2539) at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2452) locked <0x00000006802d9a58> (a org.apache.hadoop.conf.Configuration) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1028) at org.apache.spark.sql.internal.SharedState.<init>(SharedState.scala:59) at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:130) at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:130) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.SparkSession.sharedState$lzycompute(SparkSession.scala:130) locked <0x0000000680dd97b8> (a org.apache.spark.sql.SparkSession) at org.apache.spark.sql.SparkSession.sharedState(SparkSession.scala:129) at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:326) at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1115) at org.apache.spark.sql.SparkSession$$anonfun$sessionState$2.apply(SparkSession.scala:154) at org.apache.spark.sql.SparkSession$$anonfun$sessionState$2.apply(SparkSession.scala:153) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.SparkSession.sessionState$lzycompute(SparkSession.scala:153) locked <0x0000000680dd97b8> (a org.apache.spark.sql.SparkSession) at org.apache.spark.sql.SparkSession.sessionState(SparkSession.scala:150) at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:55) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.<init>(SparkSQLCLIDriver.scala:317) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:168) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) 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.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:856) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:931) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:940) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) "spark-listener-group-shared": at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2448) waiting to lock <0x0000000680d8b560> (a org.apache.hadoop.conf.Configuration) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1028) at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1078) at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2236) at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2654) at org.apache.hadoop.fs.FsUrlStreamHandlerFactory.createURLStreamHandler(FsUrlStreamHandlerFactory.java:74) at java.net.URL.getURLStreamHandler(URL.java:1255) locked <0x00000006802b6d68> (a java.lang.Object) at java.net.URL.<init>(URL.java:617) at java.net.URL.<init>(URL.java:508) at java.net.URL.<init>(URL.java:457) at javax.crypto.JceSecurity.<clinit>(JceSecurity.java:229) at javax.crypto.Cipher.getInstance(Cipher.java:518) at sun.security.ssl.JsseJce.getCipher(JsseJce.java:213) at sun.security.ssl.SSLCipher.isTransformationAvailable(SSLCipher.java:483) at sun.security.ssl.SSLCipher.<init>(SSLCipher.java:472) at sun.security.ssl.SSLCipher.<clinit>(SSLCipher.java:81) at sun.security.ssl.CipherSuite.<clinit>(CipherSuite.java:69) at sun.security.ssl.SSLContextImpl.getApplicableSupportedCipherSuites(SSLContextImpl.java:345) at sun.security.ssl.SSLContextImpl.access$100(SSLContextImpl.java:46) at sun.security.ssl.SSLContextImpl$AbstractTLSContext.<clinit>(SSLContextImpl.java:577) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:264) at java.security.Provider$Service.getImplClass(Provider.java:1703) at java.security.Provider$Service.newInstance(Provider.java:1661) at sun.security.jca.GetInstance.getInstance(GetInstance.java:236) at sun.security.jca.GetInstance.getInstance(GetInstance.java:164) at javax.net.ssl.SSLContext.getInstance(SSLContext.java:156) at org.apache.http.conn.ssl.SSLContexts.createDefault(SSLContexts.java:58) at org.apache.http.conn.ssl.SSLSocketFactory.getSocketFactory(SSLSocketFactory.java:171) at org.apache.http.impl.conn.SchemeRegistryFactory.createDefault(SchemeRegistryFactory.java:53) at org.apache.http.impl.client.AbstractHttpClient.createClientConnectionManager(AbstractHttpClient.java:266) at org.apache.http.impl.client.AbstractHttpClient.getConnectionManager(AbstractHttpClient.java:437) locked <0x0000000680dd0000> (a org.apache.http.impl.client.DefaultHttpClient) at org.apache.http.impl.client.AbstractHttpClient.createHttpContext(AbstractHttpClient.java:246) at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:771) locked <0x0000000680dd0000> (a org.apache.http.impl.client.DefaultHttpClient) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:118) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56) ... at org.apache.spark.scheduler.SparkListenerBus$class.doPostEvent(SparkListenerBus.scala:55) at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:91) at org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$super$postToAll(AsyncEventQueue.scala:92) at org.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply$mcJ$sp(AsyncEventQueue.scala:92) at org.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply(AsyncEventQueue.scala:87) at org.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply(AsyncEventQueue.scala:87) at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58) at org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:87) at org.apache.spark.scheduler.AsyncEventQueue$$anon$1$$anonfun$run$1.apply$mcV$sp(AsyncEventQueue.scala:83) at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1327) at org.apache.spark.scheduler.AsyncEventQueue$$anon$1.run(AsyncEventQueue.scala:82)
Attachments
Issue Links
- is caused by
-
SPARK-31692 Hadoop confs passed via spark config are not set in URLStream Handler Factory
- Resolved
- is related to
-
HADOOP-16159 Deadlock when using FsUrlStreamHandlerFactory
- Open
-
SPARK-31933 Found another deadlock in Spark Driver
- Open
-
SPARK-26961 Found Java-level deadlock in Spark Driver
- Resolved