Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-26961

Found Java-level deadlock in Spark Driver

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.3.0
    • 2.3.4, 2.4.2, 3.0.0
    • Spark Submit
    • None

    Description

      Our spark job usually will finish in minutes, however, we recently found it take days to run, and we can only kill it when this happened.

      An investigation show all worker container could not connect drive after start, and driver is hanging, using jstack, we found a Java-level deadlock.

       

      Jstack output for deadlock part is showing below:

       

      Found one Java-level deadlock:
      =============================
      "SparkUI-907":
      waiting to lock monitor 0x00007f387761b398 (object 0x00000005c0c1e5e0, a org.apache.hadoop.conf.Configuration),
      which is held by "ForkJoinPool-1-worker-57"
      "ForkJoinPool-1-worker-57":
      waiting to lock monitor 0x00007f3860574298 (object 0x00000005b7991168, a org.apache.spark.util.MutableURLClassLoader),
      which is held by "ForkJoinPool-1-worker-7"
      "ForkJoinPool-1-worker-7":
      waiting to lock monitor 0x00007f387761b398 (object 0x00000005c0c1e5e0, a org.apache.hadoop.conf.Configuration),
      which is held by "ForkJoinPool-1-worker-57"

      Java stack information for the threads listed above:
      ===================================================
      "SparkUI-907":
      at org.apache.hadoop.conf.Configuration.getOverlay(Configuration.java:1328)

      • waiting to lock <0x00000005c0c1e5e0> (a org.apache.hadoop.conf.Configuration)
        at org.apache.hadoop.conf.Configuration.handleDeprecation(Configuration.java:684)
        at org.apache.hadoop.conf.Configuration.get(Configuration.java:1088)
        at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1145)
        at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2363)
        at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2840)
        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 org.apache.spark.ui.JettyUtils$$anon$4.doRequest(JettyUtils.scala:176)
        at org.apache.spark.ui.JettyUtils$$anon$4.doGet(JettyUtils.scala:161)
        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.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)
        "ForkJoinPool-1-worker-57":
        at java.lang.ClassLoader.loadClass(ClassLoader.java:404)
      • waiting to lock <0x00000005b7991168> (a org.apache.spark.util.MutableURLClassLoader)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
        at org.apache.xerces.parsers.ObjectFactory.findProviderClass(Unknown Source)
        at org.apache.xerces.parsers.ObjectFactory.newInstance(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.DOMParser.<init>(Unknown Source)
        at org.apache.xerces.parsers.DOMParser.<init>(Unknown Source)
        at org.apache.xerces.jaxp.DocumentBuilderImpl.<init>(Unknown Source)
        at org.apache.xerces.jaxp.DocumentBuilderFactoryImpl.newDocumentBuilder(Unknown Source)
        at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2737)
        at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2696)
        at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2579)
      • locked <0x00000005c0c1e5e0> (a org.apache.hadoop.conf.Configuration)
        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.getClass(Configuration.java:2363)
        at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2840)
        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.URL.openStream(URL.java:1045)
        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 scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:42)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
        at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
        at scala.collection.TraversableLike$class.filterImpl(TraversableLike.scala:247)
        at scala.collection.TraversableLike$class.filter(TraversableLike.scala:259)
        at scala.collection.AbstractTraversable.filter(Traversable.scala:104)
        at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:614)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:190)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:174)
        at com.amazon.economics.spark.dq.io.SanjuroDataRegistryData$class.readInstance(SanjuroDataRegistryData.scala:16)
        at com.amazon.economics.spark.dq.metrics.DifferenceRatioMetricSuite.readInstance(DifferenceRatioMetricSuite.scala:19)
        at com.amazon.economics.spark.dq.metrics.ComparisonStandardProvider$class.loadComparisonStandardInstance(ComparisonStandardProvider.scala:21)
        at com.amazon.economics.spark.dq.metrics.DifferenceRatioMetricSuite.loadComparisonStandardInstance(DifferenceRatioMetricSuite.scala:19)
        at com.amazon.economics.spark.dq.metrics.DifferenceRatioMetricSuite.compute(DifferenceRatioMetricSuite.scala:35)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1$$anonfun$apply$2.apply(MetricsTaskRunner.scala:41)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1$$anonfun$apply$2.apply(MetricsTaskRunner.scala:39)
        at scala.util.Try$.apply(Try.scala:192)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1.apply(MetricsTaskRunner.scala:39)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1.apply(MetricsTaskRunner.scala:32)
        at scala.collection.parallel.AugmentedIterableIterator$class.map2combiner(RemainsIterator.scala:115)
        at scala.collection.parallel.immutable.ParVector$ParVectorIterator.map2combiner(ParVector.scala:62)
        at scala.collection.parallel.ParIterableLike$Map.leaf(ParIterableLike.scala:1054)
        at scala.collection.parallel.Task$$anonfun$tryLeaf$1.apply$mcV$sp(Tasks.scala:49)
        at scala.collection.parallel.Task$$anonfun$tryLeaf$1.apply(Tasks.scala:48)
        at scala.collection.parallel.Task$$anonfun$tryLeaf$1.apply(Tasks.scala:48)
        at scala.collection.parallel.Task$class.tryLeaf(Tasks.scala:51)
        at scala.collection.parallel.ParIterableLike$Map.tryLeaf(ParIterableLike.scala:1051)
        at scala.collection.parallel.AdaptiveWorkStealingTasks$WrappedTask$class.compute(Tasks.scala:152)
        at scala.collection.parallel.AdaptiveWorkStealingForkJoinTasks$WrappedTask.compute(Tasks.scala:443)
        at scala.concurrent.forkjoin.RecursiveAction.exec(RecursiveAction.java:160)
        at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
        "ForkJoinPool-1-worker-7":
        at org.apache.hadoop.conf.Configuration.getOverlay(Configuration.java:1328)
      • waiting to lock <0x00000005c0c1e5e0> (a org.apache.hadoop.conf.Configuration)
        at org.apache.hadoop.conf.Configuration.handleDeprecation(Configuration.java:684)
        at org.apache.hadoop.conf.Configuration.get(Configuration.java:1088)
        at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1145)
        at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2363)
        at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2840)
        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 sun.misc.URLClassPath$JarLoader.<init>(URLClassPath.java:812)
        at sun.misc.URLClassPath$JarLoader$3.run(URLClassPath.java:1094)
        at sun.misc.URLClassPath$JarLoader$3.run(URLClassPath.java:1091)
        at java.security.AccessController.doPrivileged(Native Method)
        at sun.misc.URLClassPath$JarLoader.getResource(URLClassPath.java:1090)
        at sun.misc.URLClassPath$JarLoader.getResource(URLClassPath.java:1050)
        at sun.misc.URLClassPath.getResource(URLClassPath.java:239)
        at java.net.URLClassLoader$1.run(URLClassLoader.java:365)
        at java.net.URLClassLoader$1.run(URLClassLoader.java:362)
        at java.security.AccessController.doPrivileged(Native Method)
        at java.net.URLClassLoader.findClass(URLClassLoader.java:361)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
      • locked <0x00000005b7991168> (a org.apache.spark.util.MutableURLClassLoader)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
        at java.lang.Class.forName0(Native Method)
        at java.lang.Class.forName(Class.java:348)
        at scala.reflect.runtime.JavaMirrors$JavaMirror.javaClass(JavaMirrors.scala:555)
        at scala.reflect.runtime.JavaMirrors$JavaMirror.tryJavaClass(JavaMirrors.scala:559)
        at scala.reflect.runtime.SymbolLoaders$PackageScope$$anonfun$lookupEntry$1.apply(SymbolLoaders.scala:137)
        at scala.reflect.runtime.SymbolLoaders$PackageScope$$anonfun$lookupEntry$1.apply(SymbolLoaders.scala:126)
        at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
        at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
        at scala.reflect.runtime.SymbolLoaders$PackageScope.syncLockSynchronized(SymbolLoaders.scala:124)
        at scala.reflect.runtime.SymbolLoaders$PackageScope.lookupEntry(SymbolLoaders.scala:126)
        at scala.reflect.internal.Types$Type.findDecl(Types.scala:971)
        at scala.reflect.internal.Types$Type.decl(Types.scala:566)
        at scala.reflect.internal.SymbolTable.openPackageModule(SymbolTable.scala:335)
        at scala.reflect.runtime.SymbolLoaders$LazyPackageType$$anonfun$complete$2.apply$mcV$sp(SymbolLoaders.scala:74)
        at scala.reflect.runtime.SymbolLoaders$LazyPackageType$$anonfun$complete$2.apply(SymbolLoaders.scala:71)
        at scala.reflect.runtime.SymbolLoaders$LazyPackageType$$anonfun$complete$2.apply(SymbolLoaders.scala:71)
        at scala.reflect.internal.SymbolTable.slowButSafeEnteringPhaseNotLaterThan(SymbolTable.scala:263)
        at scala.reflect.runtime.SymbolLoaders$LazyPackageType.complete(SymbolLoaders.scala:71)
        at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1514)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$1.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(SynchronizedSymbols.scala:174)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
        at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
        at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.gilSynchronizedIfNotThreadsafe(SynchronizedSymbols.scala:123)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$1.gilSynchronizedIfNotThreadsafe(SynchronizedSymbols.scala:174)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.info(SynchronizedSymbols.scala:127)
        at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$1.info(SynchronizedSymbols.scala:174)
        at scala.reflect.internal.Types$TypeRef.thisInfo(Types.scala:2194)
        at scala.reflect.internal.Types$TypeRef.baseClasses(Types.scala:2199)
        at scala.reflect.internal.tpe.FindMembers$FindMemberBase.<init>(FindMembers.scala:17)
        at scala.reflect.internal.tpe.FindMembers$FindMember.<init>(FindMembers.scala:219)
        at scala.reflect.internal.Types$Type.scala$reflect$internal$Types$Type$$findMemberInternal$1(Types.scala:1014)
        at scala.reflect.internal.Types$Type.findMember(Types.scala:1016)
        at scala.reflect.internal.Types$Type.memberBasedOnName(Types.scala:631)
        at scala.reflect.internal.Types$Type.member(Types.scala:600)
        at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48)
        at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
        at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
        at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
        at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
        at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:66)
        at scala.reflect.internal.Mirrors$RootsBase.staticModuleOrClass(Mirrors.scala:77)
        at scala.reflect.internal.Mirrors$RootsBase.staticModule(Mirrors.scala:161)
        at scala.reflect.internal.Mirrors$RootsBase.staticModule(Mirrors.scala:22)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$schemaFor$1$$typecreator48$1.apply(ScalaReflection.scala:726)
        at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe$lzycompute(TypeTags.scala:232)
      • locked <0x00000005c0c47770> (a scala.reflect.api.TypeTags$TypeTagImpl)
        at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe(TypeTags.scala:232)
        at org.apache.spark.sql.catalyst.ScalaReflection$class.localTypeOf(ScalaReflection.scala:839)
        at org.apache.spark.sql.catalyst.ScalaReflection$.localTypeOf(ScalaReflection.scala:39)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$schemaFor$1.apply(ScalaReflection.scala:726)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$schemaFor$1.apply(ScalaReflection.scala:715)
        at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
        at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
        at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
        at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:714)
        at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:711)
        at org.apache.spark.sql.functions$.udf(functions.scala:3356)
        at com.amazon.economics.spark.stata.port.RowTotal$.rowtotalOf(RowTotal.scala:28)
        at com.amazon.economics.spark.stata.port.RowTotal$.rowtotalInt$lzycompute(RowTotal.scala:61)
      • locked <0x00000005c0c47858> (a com.amazon.economics.spark.stata.port.RowTotal$)
        at com.amazon.economics.spark.stata.port.RowTotal$.rowtotalInt(RowTotal.scala:61)
        at com.amazon.economics.spark.stata.port.DataFrameRowtotalOps.<init>(RowTotal.scala:71)
        at com.amazon.economics.spark.stata.port.DataFrameRowtotalOps$.toDataFrameRowTotalOps(RowTotal.scala:140)
        at com.amazon.economics.spark.dq.metrics.DifferenceRatioMetricSuite$$anonfun$1.apply(DifferenceRatioMetricSuite.scala:29)
        at com.amazon.economics.spark.dq.metrics.DifferenceRatioMetricSuite$$anonfun$1.apply(DifferenceRatioMetricSuite.scala:29)
        at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
        at scala.collection.immutable.List.foldLeft(List.scala:84)
        at com.amazon.economics.spark.dq.metrics.DifferenceRatioMetricSuite.compute(DifferenceRatioMetricSuite.scala:29)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1$$anonfun$apply$2.apply(MetricsTaskRunner.scala:41)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1$$anonfun$apply$2.apply(MetricsTaskRunner.scala:39)
        at scala.util.Try$.apply(Try.scala:192)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1.apply(MetricsTaskRunner.scala:39)
        at com.amazon.economics.spark.dq.MetricsTaskRunner$$anonfun$run$1$$anonfun$1$$anonfun$apply$1.apply(MetricsTaskRunner.scala:32)
        at scala.collection.parallel.AugmentedIterableIterator$class.map2combiner(RemainsIterator.scala:115)
        at scala.collection.parallel.immutable.ParVector$ParVectorIterator.map2combiner(ParVector.scala:62)
        at scala.collection.parallel.ParIterableLike$Map.leaf(ParIterableLike.scala:1054)
        at scala.collection.parallel.Task$$anonfun$tryLeaf$1.apply$mcV$sp(Tasks.scala:49)
        at scala.collection.parallel.Task$$anonfun$tryLeaf$1.apply(Tasks.scala:48)
        at scala.collection.parallel.Task$$anonfun$tryLeaf$1.apply(Tasks.scala:48)
        at scala.collection.parallel.Task$class.tryLeaf(Tasks.scala:51)
        at scala.collection.parallel.ParIterableLike$Map.tryLeaf(ParIterableLike.scala:1051)
        at scala.collection.parallel.AdaptiveWorkStealingTasks$WrappedTask$class.compute(Tasks.scala:152)
        at scala.collection.parallel.AdaptiveWorkStealingForkJoinTasks$WrappedTask.compute(Tasks.scala:443)
        at scala.concurrent.forkjoin.RecursiveAction.exec(RecursiveAction.java:160)
        at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

      Found 1 deadlock.

       

      Attachments

        Issue Links

          Activity

            People

              ajithshetty Ajith S
              sol401430 Rong Jialei
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: