You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Ajith S (JIRA)" <ji...@apache.org> on 2019/03/01 02:19:00 UTC

[jira] [Commented] (SPARK-26587) Deadlock between SparkUI thread and Driver thread

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

Ajith S commented on SPARK-26587:
---------------------------------

Related 

https://issues.apache.org/jira/browse/SPARK-26961?focusedCommentId=16780534&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16780534

> Deadlock between SparkUI thread and Driver thread  
> ---------------------------------------------------
>
>                 Key: SPARK-26587
>                 URL: https://issues.apache.org/jira/browse/SPARK-26587
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.2.0
>         Environment: EMR 5.9.0
>            Reporter: Vitaliy Savkin
>            Priority: Major
>         Attachments: _Spark_node_hanging__Thread_dump_from_application_master.txt
>
>
> One time in a month (~1000 runs) one of our spark applications freezes at startup. jstack says that there is a deadlock. Please see locks 0x00000000802c00c0 and 0x000000008271bb98 in stacktraces below.
> {noformat}
> "Driver":
> at java.lang.Package.getSystemPackage(Package.java:540)
> - waiting to lock <0x00000000802c00c0> (a java.util.HashMap)
> at java.lang.ClassLoader.getPackage(ClassLoader.java:1625)
> at java.net.URLClassLoader.getAndVerifyPackage(URLClassLoader.java:394)
> at java.net.URLClassLoader.definePackageInternal(URLClassLoader.java:420)
> at java.net.URLClassLoader.defineClass(URLClassLoader.java:452)
> at java.net.URLClassLoader.access$100(URLClassLoader.java:74)
> at java.net.URLClassLoader$1.run(URLClassLoader.java:369)
> at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
> at java.security.AccessController.doPrivileged(Native Method)
> at java.net.URLClassLoader.findClass(URLClassLoader.java:362)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> - locked <0x0000000082789598> (a org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1)
> at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:221)
> at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:210)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
> - locked <0x0000000082789540> (a org.apache.spark.sql.internal.NonClosableMutableURLClassLoader)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> at java.lang.Class.forName0(Native Method)
> at java.lang.Class.forName(Class.java:348)
> at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:370)
> at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> at javax.xml.parsers.FactoryFinder$1.run(FactoryFinder.java:294)
> 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:120)
> at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2516)
> at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2492)
> at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2405)
> - locked <0x000000008271bb98> (a org.apache.hadoop.conf.Configuration)
> at org.apache.hadoop.conf.Configuration.get(Configuration.java:981)
> at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1031)
> at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2189)
> at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2702)
> at org.apache.hadoop.fs.FsUrlStreamHandlerFactory.createURLStreamHandler(FsUrlStreamHandlerFactory.java:74)
> at java.net.URL.getURLStreamHandler(URL.java:1142)
> at java.net.URL.<init>(URL.java:599)
> at java.net.URL.<init>(URL.java:490)
> at java.net.URL.<init>(URL.java:439)
> 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:979)
> at java.net.URLClassLoader.getResourceAsStream(URLClassLoader.java:238)
> at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:216)
> at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:210)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
> - locked <0x0000000082789540> (a org.apache.spark.sql.internal.NonClosableMutableURLClassLoader)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:262)
> at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:362)
> at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:266)
> at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:66)
> - locked <0x000000008302a120> (a org.apache.spark.sql.hive.HiveExternalCatalog)
> at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:65)
> at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply$mcZ$sp(HiveExternalCatalog.scala:194)
> at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:194)
> at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:194)
> at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97)
> - locked <0x000000008302a120> (a org.apache.spark.sql.hive.HiveExternalCatalog)
> at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:193)
> at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:105)
> - locked <0x0000000082d8bb80> (a org.apache.spark.sql.internal.SharedState)
> at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:93)
> at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:39)
> at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog$lzycompute(HiveSessionStateBuilder.scala:54)
> - locked <0x000000008302a180> (a org.apache.spark.sql.hive.HiveSessionStateBuilder)
> at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog(HiveSessionStateBuilder.scala:52)
> at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog(HiveSessionStateBuilder.scala:35)
> at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:289)
> at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1050)
> at org.apache.spark.sql.SparkSession$$anonfun$sessionState$2.apply(SparkSession.scala:130)
> at org.apache.spark.sql.SparkSession$$anonfun$sessionState$2.apply(SparkSession.scala:130)
> at scala.Option.getOrElse(Option.scala:121)
> at org.apache.spark.sql.SparkSession.sessionState$lzycompute(SparkSession.scala:129)
> - locked <0x0000000082d8bb40> (a org.apache.spark.sql.SparkSession)
> at org.apache.spark.sql.SparkSession.sessionState(SparkSession.scala:126)
> at org.apache.spark.sql.SparkSession$Builder$$anonfun$getOrCreate$5.apply(SparkSession.scala:938)
> at org.apache.spark.sql.SparkSession$Builder$$anonfun$getOrCreate$5.apply(SparkSession.scala:938)
> at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
> at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
> at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
> at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
> at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
> at org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:938)
> - locked <0x00000000805d7f48> (a org.apache.spark.sql.SparkSession$)
> - locked <0x00000000805d7f68> (a org.apache.spark.sql.SparkSession$Builder)
> at com.BatchPartitioner$.run(BatchPartitioner.scala:72)
> at com.BatchPartitioner$.main(BatchPartitioner.scala:59)
> at com.BatchPartitioner.main(BatchPartitioner.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.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:635)
> {noformat}
> {noformat}
> "SparkUI-43":
> at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2401)
> - waiting to lock <0x000000008271bb98> (a org.apache.hadoop.conf.Configuration)
> at org.apache.hadoop.conf.Configuration.get(Configuration.java:981)
> at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1031)
> at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2189)
> at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2702)
> at org.apache.hadoop.fs.FsUrlStreamHandlerFactory.createURLStreamHandler(FsUrlStreamHandlerFactory.java:74)
> at java.net.URL.getURLStreamHandler(URL.java:1142)
> at java.net.URL.<init>(URL.java:420)
> at java.net.URL.<init>(URL.java:310)
> at java.net.URL.<init>(URL.java:333)
> at sun.net.www.ParseUtil.fileToEncodedURL(ParseUtil.java:272)
> at java.lang.Package$1.run(Package.java:579)
> at java.lang.Package$1.run(Package.java:570)
> at java.security.AccessController.doPrivileged(Native Method)
> at java.lang.Package.defineSystemPackage(Package.java:570)
> at java.lang.Package.getSystemPackage(Package.java:546)
> - locked <0x00000000802c00c0> (a java.util.HashMap)
> at java.lang.Package.getPackage(Package.java:336)
> at java.lang.Class.getPackage(Class.java:796)
> at com.fasterxml.jackson.databind.introspect.BasicClassIntrospector._isStdJDKCollection(BasicClassIntrospector.java:257)
> at com.fasterxml.jackson.databind.introspect.BasicClassIntrospector._findStdJdkCollectionDesc(BasicClassIntrospector.java:277)
> at com.fasterxml.jackson.databind.introspect.BasicClassIntrospector.forSerialization(BasicClassIntrospector.java:88)
> at com.fasterxml.jackson.databind.introspect.BasicClassIntrospector.forSerialization(BasicClassIntrospector.java:15)
> at com.fasterxml.jackson.databind.SerializationConfig.introspect(SerializationConfig.java:789)
> at com.fasterxml.jackson.databind.ser.BeanSerializerFactory.createSerializer(BeanSerializerFactory.java:134)
> at com.fasterxml.jackson.databind.SerializerProvider._createUntypedSerializer(SerializerProvider.java:1203)
> - locked <0x0000000082913c68> (a com.fasterxml.jackson.databind.ser.SerializerCache)
> at com.fasterxml.jackson.databind.SerializerProvider._createAndCacheUntypedSerializer(SerializerProvider.java:1157)
> at com.fasterxml.jackson.databind.SerializerProvider.findValueSerializer(SerializerProvider.java:481)
> at com.fasterxml.jackson.databind.SerializerProvider.findTypedValueSerializer(SerializerProvider.java:679)
> at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider.serializeValue(DefaultSerializerProvider.java:107)
> at com.fasterxml.jackson.databind.ObjectMapper.writeValue(ObjectMapper.java:2383)
> at com.fasterxml.jackson.core.base.GeneratorBase.writeObject(GeneratorBase.java:324)
> at com.fasterxml.jackson.core.JsonGenerator.writeObjectField(JsonGenerator.java:1415)
> at com.codahale.metrics.json.MetricsModule$MetricRegistrySerializer.serialize(MetricsModule.java:186)
> at com.codahale.metrics.json.MetricsModule$MetricRegistrySerializer.serialize(MetricsModule.java:171)
> at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider.serializeValue(DefaultSerializerProvider.java:130)
> at com.fasterxml.jackson.databind.ObjectMapper._configAndWriteValue(ObjectMapper.java:3559)
> at com.fasterxml.jackson.databind.ObjectMapper.writeValueAsString(ObjectMapper.java:2927)
> at org.apache.spark.metrics.sink.MetricsServlet.getMetricsSnapshot(MetricsServlet.scala:59)
> at org.apache.spark.metrics.sink.MetricsServlet$$anonfun$getHandlers$1.apply(MetricsServlet.scala:54)
> at org.apache.spark.metrics.sink.MetricsServlet$$anonfun$getHandlers$1.apply(MetricsServlet.scala:54)
> at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
> at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
> at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
> at org.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:845)
> at org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:583)
> at org.spark_project.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
> at org.spark_project.jetty.servlet.ServletHandler.doScope(ServletHandler.java:511)
> at org.spark_project.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
> at org.spark_project.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
> at org.spark_project.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)
> at org.spark_project.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
> at org.spark_project.jetty.server.Server.handle(Server.java:524)
> at org.spark_project.jetty.server.HttpChannel.handle(HttpChannel.java:319)
> at org.spark_project.jetty.server.HttpConnection.onFillable(HttpConnection.java:253)
> at org.spark_project.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:273)
> at org.spark_project.jetty.io.FillInterest.fillable(FillInterest.java:95)
> at org.spark_project.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
> at org.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
> at org.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
> at org.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
> at org.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
> at org.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
> at java.lang.Thread.run(Thread.java:748)
> {noformat}
> It happens only with an application working with Hive. Others seem to be not affected.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org