You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Tommy S (JIRA)" <ji...@apache.org> on 2018/07/25 23:10:00 UTC

[jira] [Updated] (SPARK-24921) SparkStreaming steadily increasing job generation delay due to apparent URLClassLoader contention

     [ https://issues.apache.org/jira/browse/SPARK-24921?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Tommy S updated SPARK-24921:
----------------------------
    Component/s: Web UI

> SparkStreaming steadily increasing job generation delay due to apparent URLClassLoader contention
> -------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-24921
>                 URL: https://issues.apache.org/jira/browse/SPARK-24921
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core, Web UI
>    Affects Versions: 2.3.1
>            Reporter: Tommy S
>            Priority: Major
>
> I'm seeing an issue where the job generation time of my spark streaming job is steadily increasing after some time.
> Looking at the thread dumps I see that the JobGenerator thread is BLOCKED waiting for URLClassPath.getLoader synchronized method:
> {noformat}
> "JobGenerator" #153 daemon prio=5 os_prio=0 tid=0x0000000002dad800 nid=0x253c waiting for monitor entry [0x00007f4b311c2000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at sun.misc.URLClassPath.getNextLoader(URLClassPath.java:469)
>         - waiting to lock <0x00007f4be023f940> (a sun.misc.URLClassPath)
>         at sun.misc.URLClassPath.findResource(URLClassPath.java:214)
>         at java.net.URLClassLoader$2.run(URLClassLoader.java:569)
>         at java.net.URLClassLoader$2.run(URLClassLoader.java:567)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findResource(URLClassLoader.java:566)
>         at java.lang.ClassLoader.getResource(ClassLoader.java:1096)
>         at java.lang.ClassLoader.getResource(ClassLoader.java:1091)
>         at java.net.URLClassLoader.getResourceAsStream(URLClassLoader.java:232)
>         at java.lang.Class.getResourceAsStream(Class.java:2223)
>         at org.apache.spark.util.ClosureCleaner$.getClassReader(ClosureCleaner.scala:40)
>         at org.apache.spark.util.ClosureCleaner$.getInnerClosureClasses(ClosureCleaner.scala:84)
>         at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:224)
>         at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:159)
>         at org.apache.spark.SparkContext.clean(SparkContext.scala:2299)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKeyWithClassTag$1.apply(PairRDDFunctions.scala:89)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKeyWithClassTag$1.apply(PairRDDFunctions.scala:77)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>         at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
>         at org.apache.spark.rdd.PairRDDFunctions.combineByKeyWithClassTag(PairRDDFunctions.scala:77)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKey$1.apply(PairRDDFunctions.scala:119)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKey$1.apply(PairRDDFunctions.scala:119)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>         at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
>         at org.apache.spark.rdd.PairRDDFunctions.combineByKey(PairRDDFunctions.scala:117)
>         at org.apache.spark.streaming.dstream.ShuffledDStream.compute(ShuffledDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.TransformedDStream.createRDDWithLocalProperties(TransformedDStream.scala:65)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         - locked <0x00007f4bf91ca1c0> (a org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> "metrics-statsd-reporter-thread-1" #158 daemon prio=5 os_prio=0 tid=0x00007f597133a220 nid=0x2540 waiting on condition [0x00007f4b314c8000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f4bf71ba708> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> "JobGenerator" #153 daemon prio=5 os_prio=0 tid=0x0000000002dad800 nid=0x253c waiting for monitor entry [0x00007f4b311c2000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at sun.misc.URLClassPath.getNextLoader(URLClassPath.java:469)
>         - waiting to lock <0x00007f4be023f940> (a sun.misc.URLClassPath)
>         at sun.misc.URLClassPath.findResource(URLClassPath.java:214)
>         at java.net.URLClassLoader$2.run(URLClassLoader.java:569)
>         at java.net.URLClassLoader$2.run(URLClassLoader.java:567)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findResource(URLClassLoader.java:566)
>         at java.lang.ClassLoader.getResource(ClassLoader.java:1096)
>         at java.lang.ClassLoader.getResource(ClassLoader.java:1091)
>         at java.net.URLClassLoader.getResourceAsStream(URLClassLoader.java:232)
>         at java.lang.Class.getResourceAsStream(Class.java:2223)
>         at org.apache.spark.util.ClosureCleaner$.getClassReader(ClosureCleaner.scala:40)
>         at org.apache.spark.util.ClosureCleaner$.getInnerClosureClasses(ClosureCleaner.scala:84)
>         at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:224)
>         at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:159)
>         at org.apache.spark.SparkContext.clean(SparkContext.scala:2299)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKeyWithClassTag$1.apply(PairRDDFunctions.scala:89)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKeyWithClassTag$1.apply(PairRDDFunctions.scala:77)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>         at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
>         at org.apache.spark.rdd.PairRDDFunctions.combineByKeyWithClassTag(PairRDDFunctions.scala:77)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKey$1.apply(PairRDDFunctions.scala:119)
>         at org.apache.spark.rdd.PairRDDFunctions$$anonfun$combineByKey$1.apply(PairRDDFunctions.scala:119)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>         at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
>         at org.apache.spark.rdd.PairRDDFunctions.combineByKey(PairRDDFunctions.scala:117)
>         at org.apache.spark.streaming.dstream.ShuffledDStream.compute(ShuffledDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.TransformedDStream.createRDDWithLocalProperties(TransformedDStream.scala:65)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MapValuedDStream.compute(MapValuedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.ShuffledDStream.compute(ShuffledDStream.scala:41)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.TransformedDStream.createRDDWithLocalProperties(TransformedDStream.scala:65)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MapValuedDStream.compute(MapValuedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.TransformedDStream.createRDDWithLocalProperties(TransformedDStream.scala:65)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.TransformedDStream.createRDDWithLocalProperties(TransformedDStream.scala:65)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MapValuedDStream.compute(MapValuedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.TransformedDStream$$anonfun$6.apply(TransformedDStream.scala:42)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:42)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.TransformedDStream.createRDDWithLocalProperties(TransformedDStream.scala:65)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.MappedDStream.compute(MappedDStream.scala:36)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1$$anonfun$apply$7.apply(DStream.scala:342)
>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1$$anonfun$1.apply(DStream.scala:341)
>         at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:336)
>         at org.apache.spark.streaming.dstream.DStream$$anonfun$getOrCompute$1.apply(DStream.scala:334)
>         at scala.Option.orElse(Option.scala:289)
>         at org.apache.spark.streaming.dstream.DStream.getOrCompute(DStream.scala:331)
>         at org.apache.spark.streaming.dstream.ForEachDStream.generateJob(ForEachDStream.scala:48)
>         at org.apache.spark.streaming.DStreamGraph$$anonfun$1.apply(DStreamGraph.scala:122)
>         at org.apache.spark.streaming.DStreamGraph$$anonfun$1.apply(DStreamGraph.scala:121)
>         at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
>         at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
>         at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>         at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
>         at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
>         at scala.collection.AbstractTraversable.flatMap(Traversable.scala:104)
>         at org.apache.spark.streaming.DStreamGraph.generateJobs(DStreamGraph.scala:121)
>         - locked <0x00007f4be8b1bca0> (a org.apache.spark.streaming.DStreamGraph)
>         at org.apache.spark.streaming.scheduler.JobGenerator$$anonfun$3.apply(JobGenerator.scala:249)
>         at org.apache.spark.streaming.scheduler.JobGenerator$$anonfun$3.apply(JobGenerator.scala:247)
>         at scala.util.Try$.apply(Try.scala:192)
>         at org.apache.spark.streaming.scheduler.JobGenerator.generateJobs(JobGenerator.scala:247)
>         at org.apache.spark.streaming.scheduler.JobGenerator.org$apache$spark$streaming$scheduler$JobGenerator$$processEvent(JobGenerator.scala:183)
>         at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:89)
>         at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:88)
>         at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48){noformat}
> The thread that is holding the lock is SparkUI.
> {noformat}
> "SparkUI-11682" #11682 daemon prio=5 os_prio=0 tid=0x00007f5972cd3610 nid=0x2d5f runnable [0x00007f4b1cb59000]
>    java.lang.Thread.State: RUNNABLE
>         at sun.misc.URLClassPath.getLoader(URLClassPath.java:494)
>         - locked <0x00007f4be023f940> (a sun.misc.URLClassPath)
>         at sun.misc.URLClassPath.getNextLoader(URLClassPath.java:484)
>         - locked <0x00007f4be023f940> (a sun.misc.URLClassPath)
>         at sun.misc.URLClassPath.findResource(URLClassPath.java:214)
>         at java.net.URLClassLoader$2.run(URLClassLoader.java:569)
>         at java.net.URLClassLoader$2.run(URLClassLoader.java:567)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findResource(URLClassLoader.java:566)
>         at java.lang.ClassLoader.getResource(ClassLoader.java:1096)
>         at java.lang.ClassLoader.getResource(ClassLoader.java:1091)
>         at java.net.URLClassLoader.getResourceAsStream(URLClassLoader.java:232)
>         at org.apache.xerces.parsers.SecuritySupport$6.run(Unknown Source)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at org.apache.xerces.parsers.SecuritySupport.getResourceAsStream(Unknown Source)
>         at org.apache.xerces.parsers.ObjectFactory.findJarServiceProvider(Unknown Source)
>         at org.apache.xerces.parsers.ObjectFactory.createObject(Unknown Source)
>         at org.apache.xerces.parsers.ObjectFactory.createObject(Unknown Source)
>         at org.apache.xerces.parsers.SAXParser.<init>(Unknown Source)
>         at org.apache.xerces.parsers.SAXParser.<init>(Unknown Source)
>         at org.apache.xerces.jaxp.SAXParserImpl$JAXPSAXParser.<init>(Unknown Source)
>         at org.apache.xerces.jaxp.SAXParserImpl.<init>(Unknown Source)
>         at org.apache.xerces.jaxp.SAXParserFactoryImpl.newSAXParser(Unknown Source)
>         at scala.xml.factory.XMLLoader$class.parser(XMLLoader.scala:30)
>         at scala.xml.XML$.parser(XML.scala:60)
>         at scala.xml.factory.XMLLoader$class.loadString(XMLLoader.scala:60)
>         at scala.xml.XML$.loadString(XML.scala:60)
>         at org.apache.spark.ui.UIUtils$.makeDescription(UIUtils.scala:451)
>         at org.apache.spark.ui.jobs.JobDataSource.org$apache$spark$ui$jobs$JobDataSource$$jobRow(AllJobsPage.scala:438)
>         at org.apache.spark.ui.jobs.JobDataSource$$anonfun$24.apply(AllJobsPage.scala:414)
>         at org.apache.spark.ui.jobs.JobDataSource$$anonfun$24.apply(AllJobsPage.scala:414)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>         at scala.collection.mutable.ListBuffer.foreach(ListBuffer.scala:45)
>         at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>         at org.apache.spark.ui.jobs.JobDataSource.<init>(AllJobsPage.scala:414)
>         at org.apache.spark.ui.jobs.JobPagedTable.<init>(AllJobsPage.scala:506)
>         at org.apache.spark.ui.jobs.AllJobsPage.jobsTable(AllJobsPage.scala:248)
>         at org.apache.spark.ui.jobs.AllJobsPage.render(AllJobsPage.scala:297)
>         at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
>         at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
>         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:848)
>         at org.spark_project.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1772)
>         at org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.doFilter(AmIpFilter.java:171)
>         at org.spark_project.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
>         at org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:582)
>         at org.spark_project.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
>         at org.spark_project.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
>         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.gzip.GzipHandler.handle(GzipHandler.java:493)
>         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:534)
>         at org.spark_project.jetty.server.HttpChannel.handle(HttpChannel.java:320)
>         at org.spark_project.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
>         at org.spark_project.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
>         at org.spark_project.jetty.io.FillInterest.fillable(FillInterest.java:108)
>         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}
> Also there seem to be about 60 SparkUI threads all blocked on the same lock
> Any ideas why this might be happening? It seems that the threads are using the classLoader to do some object mapping to/from XML. Ideally I'd imagine it would be better if the SparkUI doesn't block the JobGenerator since we can live with the SparkUI threads contending for resource, but not so much the job slowing down.
> We're running this in AWS EMR (v. emr-5.16.0) with Spark 2.3.1



--
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