We'll try to run a build tomorrow AM.
On Mon, Jul 14, 2014 at 7:22 PM, Patrick Wendell <pwend...@gmail.com> wrote: > Andrew and Gary, > > Would you guys be able to test > https://github.com/apache/spark/pull/1409/files and see if it solves > your problem? > > - Patrick > > On Mon, Jul 14, 2014 at 4:18 PM, Andrew Ash <and...@andrewash.com> wrote: > > I observed a deadlock here when using the AvroInputFormat as well. The > > short of the issue is that there's one configuration object per JVM, but > > multiple threads, one for each task. If each thread attempts to add a > > configuration option to the Configuration object at once you get issues > > because HashMap isn't thread safe. > > > > More details to come tonight. Thanks! > > On Jul 14, 2014 4:11 PM, "Nishkam Ravi" <nr...@cloudera.com> wrote: > > > >> HI Patrick, I'm not aware of another place where the access happens, but > >> it's possible that it does. The original fix synchronized on the > >> broadcastConf object and someone reported the same exception. > >> > >> > >> On Mon, Jul 14, 2014 at 3:57 PM, Patrick Wendell <pwend...@gmail.com> > >> wrote: > >> > >> > Hey Nishkam, > >> > > >> > Aaron's fix should prevent two concurrent accesses to getJobConf (and > >> > the Hadoop code therein). But if there is code elsewhere that tries to > >> > mutate the configuration, then I could see how we might still have the > >> > ConcurrentModificationException. > >> > > >> > I looked at your patch for HADOOP-10456 and the only example you give > >> > is of the data being accessed inside of getJobConf. Is it accessed > >> > somewhere else too from Spark that you are aware of? > >> > > >> > https://issues.apache.org/jira/browse/HADOOP-10456 > >> > > >> > - Patrick > >> > > >> > On Mon, Jul 14, 2014 at 3:28 PM, Nishkam Ravi <nr...@cloudera.com> > >> wrote: > >> > > Hi Aaron, I'm not sure if synchronizing on an arbitrary lock object > >> would > >> > > help. I suspect we will start seeing the > >> ConcurrentModificationException > >> > > again. The right fix has gone into Hadoop through 10456. > >> Unfortunately, I > >> > > don't have any bright ideas on how to synchronize this at the Spark > >> level > >> > > without the risk of deadlocks. > >> > > > >> > > > >> > > On Mon, Jul 14, 2014 at 3:07 PM, Aaron Davidson <ilike...@gmail.com > > > >> > wrote: > >> > > > >> > >> The full jstack would still be useful, but our current working > theory > >> is > >> > >> that this is due to the fact that Configuration#loadDefaults goes > >> > through > >> > >> every Configuration object that was ever created (via > >> > >> Configuration.REGISTRY) and locks it, thus introducing a dependency > >> from > >> > >> new Configuration to old, otherwise unrelated, Configuration > objects > >> > that > >> > >> our locking did not anticipate. > >> > >> > >> > >> I have created https://github.com/apache/spark/pull/1409 to > hopefully > >> > fix > >> > >> this bug. > >> > >> > >> > >> > >> > >> On Mon, Jul 14, 2014 at 2:44 PM, Patrick Wendell < > pwend...@gmail.com> > >> > >> wrote: > >> > >> > >> > >> > Hey Cody, > >> > >> > > >> > >> > This Jstack seems truncated, would you mind giving the entire > stack > >> > >> > trace? For the second thread, for instance, we can't see where > the > >> > >> > lock is being acquired. > >> > >> > > >> > >> > - Patrick > >> > >> > > >> > >> > On Mon, Jul 14, 2014 at 1:42 PM, Cody Koeninger > >> > >> > <cody.koenin...@mediacrossing.com> wrote: > >> > >> > > Hi all, just wanted to give a heads up that we're seeing a > >> > reproducible > >> > >> > > deadlock with spark 1.0.1 with 2.3.0-mr1-cdh5.0.2 > >> > >> > > > >> > >> > > If jira is a better place for this, apologies in advance - > figured > >> > >> > talking > >> > >> > > about it on the mailing list was friendlier than randomly > >> > (re)opening > >> > >> > jira > >> > >> > > tickets. > >> > >> > > > >> > >> > > I know Gary had mentioned some issues with 1.0.1 on the mailing > >> > list, > >> > >> > once > >> > >> > > we got a thread dump I wanted to follow up. > >> > >> > > > >> > >> > > The thread dump shows the deadlock occurs in the synchronized > >> block > >> > of > >> > >> > code > >> > >> > > that was changed in HadoopRDD.scala, for the Spark-1097 issue > >> > >> > > > >> > >> > > Relevant portions of the thread dump are summarized below, we > can > >> > >> provide > >> > >> > > the whole dump if it's useful. > >> > >> > > > >> > >> > > Found one Java-level deadlock: > >> > >> > > ============================= > >> > >> > > "Executor task launch worker-1": > >> > >> > > waiting to lock monitor 0x00007f250400c520 (object > >> > >> 0x00000000fae7dc30, > >> > >> > a > >> > >> > > org.apache.hadoop.co > >> > >> > > nf.Configuration), > >> > >> > > which is held by "Executor task launch worker-0" > >> > >> > > "Executor task launch worker-0": > >> > >> > > waiting to lock monitor 0x00007f2520495620 (object > >> > >> 0x00000000faeb4fc8, > >> > >> > a > >> > >> > > java.lang.Class), > >> > >> > > which is held by "Executor task launch worker-1" > >> > >> > > > >> > >> > > > >> > >> > > "Executor task launch worker-1": > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.conf.Configuration.reloadConfiguration(Configuration.java:791) > >> > >> > > - waiting to lock <0x00000000fae7dc30> (a > >> > >> > > org.apache.hadoop.conf.Configuration) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.conf.Configuration.addDefaultResource(Configuration.java:690) > >> > >> > > - locked <0x00000000faca6ff8> (a java.lang.Class for > >> > >> > > org.apache.hadoop.conf.Configurati > >> > >> > > on) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.hdfs.HdfsConfiguration.<clinit>(HdfsConfiguration.java:34) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.hdfs.DistributedFileSystem.<clinit>(DistributedFileSystem.java:110 > >> > >> > > ) > >> > >> > > at > >> > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native > >> > >> > > Method) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl. > >> > >> > > java:57) > >> > >> > > at > >> > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native > >> > >> > > Method) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl. > >> > >> > > java:57) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAcces > >> > >> > > sorImpl.java:45) > >> > >> > > at > >> > >> > java.lang.reflect.Constructor.newInstance(Constructor.java:525) > >> > >> > > at java.lang.Class.newInstance0(Class.java:374) > >> > >> > > at java.lang.Class.newInstance(Class.java:327) > >> > >> > > at > >> > >> > java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:373) > >> > >> > > at > java.util.ServiceLoader$1.next(ServiceLoader.java:445) > >> > >> > > at > >> > >> > > > >> > org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2364) > >> > >> > > - locked <0x00000000faeb4fc8> (a java.lang.Class for > >> > >> > > org.apache.hadoop.fs.FileSystem) > >> > >> > > at > >> > >> > > > >> > >> > >> org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2375) > >> > >> > > at > >> > >> > > > >> > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2392) > >> > >> > > at > >> > >> org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:89) > >> > >> > > at > >> > >> > > > >> > > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2431) > >> > >> > > at > >> > >> > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2413) > >> > >> > > at > >> org.apache.hadoop.fs.FileSystem.get(FileSystem.java:368) > >> > >> > > at > >> org.apache.hadoop.fs.FileSystem.get(FileSystem.java:167) > >> > >> > > at > >> > >> > > > >> > org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:587) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:315) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:288) > >> > >> > > at > >> > >> > > > >> > > org.apache.spark.SparkContext$$anonfun$22.apply(SparkContext.scala:546) > >> > >> > > at > >> > >> > > > >> > > org.apache.spark.SparkContext$$anonfun$22.apply(SparkContext.scala:546) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$1.apply(HadoopRDD.scala:145) > >> > >> > > > >> > >> > > > >> > >> > > > >> > >> > > ...elided... > >> > >> > > > >> > >> > > > >> > >> > > "Executor task launch worker-0" daemon prio=10 > >> > tid=0x0000000001e71800 > >> > >> > > nid=0x2d97 waiting for monitor entry [0x00007f24d2bf1000] > >> > >> > > java.lang.Thread.State: BLOCKED (on object monitor) > >> > >> > > at > >> > >> > > > >> > org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2362) > >> > >> > > - waiting to lock <0x00000000faeb4fc8> (a > java.lang.Class > >> > for > >> > >> > > org.apache.hadoop.fs.FileSystem) > >> > >> > > at > >> > >> > > > >> > >> > >> org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2375) > >> > >> > > at > >> > >> > > > >> > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2392) > >> > >> > > at > >> > >> org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:89) > >> > >> > > at > >> > >> > > > >> > > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2431) > >> > >> > > at > >> > >> > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2413) > >> > >> > > at > >> org.apache.hadoop.fs.FileSystem.get(FileSystem.java:368) > >> > >> > > at > >> org.apache.hadoop.fs.FileSystem.get(FileSystem.java:167) > >> > >> > > at > >> > >> > > > >> > org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:587) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:315) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:288) > >> > >> > > at > >> > >> > > > >> > > org.apache.spark.SparkContext$$anonfun$22.apply(SparkContext.scala:546) > >> > >> > > at > >> > >> > > > >> > > org.apache.spark.SparkContext$$anonfun$22.apply(SparkContext.scala:546) > >> > >> > > at > >> > >> > > > >> > >> > > >> > >> > >> > > >> > org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$1.apply(HadoopRDD.scala:145) > >> > >> > > >> > >> > >> > > >> >