Okay I think I've isolated this a bit more. Let's discuss over on the JIRA:
https://issues.apache.org/jira/browse/SPARK-2075 On Sun, Jun 8, 2014 at 1:16 PM, Paul Brown <p...@mult.ifario.us> wrote: > > Hi, Patrick -- > > Java 7 on the development machines: > > » java -version > 1 ↵ > java version "1.7.0_51" > Java(TM) SE Runtime Environment (build 1.7.0_51-b13) > Java HotSpot(TM) 64-Bit Server VM (build 24.51-b03, mixed mode) > > > And on the deployed boxes: > > $ java -version > java version "1.7.0_55" > OpenJDK Runtime Environment (IcedTea 2.4.7) (7u55-2.4.7-1ubuntu1) > OpenJDK 64-Bit Server VM (build 24.51-b03, mixed mode) > > > Also, "unzip -l" in place of "jar tvf" gives the same results, so I don't > think it's an issue with jar not reporting the files. Also, the classes do > get correctly packaged into the uberjar: > > unzip -l /target/[deleted]-driver.jar | grep 'rdd/RDD' | grep 'saveAs' > 1519 06-08-14 12:05 > org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$1.class > 1560 06-08-14 12:05 > org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$2.class > > > Best. > -- Paul > > — > p...@mult.ifario.us | Multifarious, Inc. | http://mult.ifario.us/ > > > On Sun, Jun 8, 2014 at 1:02 PM, Patrick Wendell <pwend...@gmail.com> wrote: >> >> Paul, >> >> Could you give the version of Java that you are building with and the >> version of Java you are running with? Are they the same? >> >> Just off the cuff, I wonder if this is related to: >> https://issues.apache.org/jira/browse/SPARK-1520 >> >> If it is, it could appear that certain functions are not in the jar >> because they go beyond the extended zip boundary `jar tvf` won't list >> them. >> >> - Patrick >> >> On Sun, Jun 8, 2014 at 12:45 PM, Paul Brown <p...@mult.ifario.us> wrote: >> > Moving over to the dev list, as this isn't a user-scope issue. >> > >> > I just ran into this issue with the missing saveAsTestFile, and here's a >> > little additional information: >> > >> > - Code ported from 0.9.1 up to 1.0.0; works with local[n] in both cases. >> > - Driver built as an uberjar via Maven. >> > - Deployed to smallish EC2 cluster in standalone mode (S3 storage) with >> > Spark 1.0.0-hadoop1 downloaded from Apache. >> > >> > Given that it functions correctly in local mode but not in a standalone >> > cluster, this suggests to me that the issue is in a difference between >> > the >> > Maven version and the hadoop1 version. >> > >> > In the spirit of taking the computer at its word, we can just have a >> > look >> > in the JAR files. Here's what's in the Maven dep as of 1.0.0: >> > >> > jar tvf >> > >> > ~/.m2/repository/org/apache/spark/spark-core_2.10/1.0.0/spark-core_2.10-1.0.0.jar >> > | grep 'rdd/RDD' | grep 'saveAs' >> > 1519 Mon May 26 13:57:58 PDT 2014 >> > org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$1.class >> > 1560 Mon May 26 13:57:58 PDT 2014 >> > org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$2.class >> > >> > >> > And here's what's in the hadoop1 distribution: >> > >> > jar tvf spark-assembly-1.0.0-hadoop1.0.4.jar| grep 'rdd/RDD' | grep >> > 'saveAs' >> > >> > >> > I.e., it's not there. It is in the hadoop2 distribution: >> > >> > jar tvf spark-assembly-1.0.0-hadoop2.2.0.jar| grep 'rdd/RDD' | grep >> > 'saveAs' >> > 1519 Mon May 26 07:29:54 PDT 2014 >> > org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$1.class >> > 1560 Mon May 26 07:29:54 PDT 2014 >> > org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$2.class >> > >> > >> > So something's clearly broken with the way that the distribution >> > assemblies >> > are created. >> > >> > FWIW and IMHO, the "right" way to publish the hadoop1 and hadoop2 >> > flavors >> > of Spark to Maven Central would be as *entirely different* artifacts >> > (spark-core-h1, spark-core-h2). >> > >> > Logged as SPARK-2075 <https://issues.apache.org/jira/browse/SPARK-2075>. >> > >> > Cheers. >> > -- Paul >> > >> > >> > >> > -- >> > p...@mult.ifario.us | Multifarious, Inc. | http://mult.ifario.us/ >> > >> > >> > On Fri, Jun 6, 2014 at 2:45 AM, HenriV <henri.vanh...@vdab.be> wrote: >> > >> >> I'm experiencing the same error while upgrading from 0.9.1 to 1.0.0. >> >> Im using google compute engine and cloud storage. but saveAsTextFile is >> >> returning errors while saving in the cloud or saving local. When i >> >> start a >> >> job in the cluster i do get an error but after this error it keeps on >> >> running fine untill the saveAsTextFile. ( I don't know if the two are >> >> connected) >> >> >> >> -----------Error at job startup------- >> >> ERROR metrics.MetricsSystem: Sink class >> >> org.apache.spark.metrics.sink.MetricsServlet cannot be instantialized >> >> java.lang.reflect.InvocationTargetException >> >> at >> >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native >> >> Method) >> >> at >> >> >> >> >> >> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) >> >> at >> >> >> >> >> >> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) >> >> at >> >> java.lang.reflect.Constructor.newInstance(Constructor.java:526) >> >> at >> >> >> >> >> >> org.apache.spark.metrics.MetricsSystem$$anonfun$registerSinks$1.apply(MetricsSystem.scala:136) >> >> at >> >> >> >> >> >> org.apache.spark.metrics.MetricsSystem$$anonfun$registerSinks$1.apply(MetricsSystem.scala:130) >> >> at >> >> >> >> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) >> >> at >> >> >> >> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) >> >> at >> >> >> >> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) >> >> at >> >> scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) >> >> at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) >> >> at >> >> >> >> >> >> org.apache.spark.metrics.MetricsSystem.registerSinks(MetricsSystem.scala:130) >> >> at >> >> org.apache.spark.metrics.MetricsSystem.<init>(MetricsSystem.scala:84) >> >> at >> >> >> >> >> >> org.apache.spark.metrics.MetricsSystem$.createMetricsSystem(MetricsSystem.scala:167) >> >> at org.apache.spark.SparkEnv$.create(SparkEnv.scala:230) >> >> at org.apache.spark.SparkContext.<init>(SparkContext.scala:202) >> >> at Hello$.main(Hello.scala:101) >> >> at Hello.main(Hello.scala) >> >> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) >> >> at >> >> >> >> >> >> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) >> >> at >> >> >> >> >> >> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) >> >> at java.lang.reflect.Method.invoke(Method.java:606) >> >> at sbt.Run.invokeMain(Run.scala:72) >> >> at sbt.Run.run0(Run.scala:65) >> >> at sbt.Run.sbt$Run$$execute$1(Run.scala:54) >> >> at sbt.Run$$anonfun$run$1.apply$mcV$sp(Run.scala:58) >> >> at sbt.Run$$anonfun$run$1.apply(Run.scala:58) >> >> at sbt.Run$$anonfun$run$1.apply(Run.scala:58) >> >> at sbt.Logger$$anon$4.apply(Logger.scala:90) >> >> at sbt.TrapExit$App.run(TrapExit.scala:244) >> >> at java.lang.Thread.run(Thread.java:744) >> >> Caused by: java.lang.NoSuchMethodError: >> >> com.fasterxml.jackson.core.JsonFactory.requiresPropertyOrdering()Z >> >> at >> >> >> >> com.fasterxml.jackson.databind.ObjectMapper.<init>(ObjectMapper.java:445) >> >> at >> >> >> >> com.fasterxml.jackson.databind.ObjectMapper.<init>(ObjectMapper.java:366) >> >> at >> >> >> >> >> >> org.apache.spark.metrics.sink.MetricsServlet.<init>(MetricsServlet.scala:45) >> >> ... 31 more >> >> >> >> then it runs fine till i get to saveAsTextFile >> >> >> >> 14/06/06 09:05:12 INFO scheduler.TaskSetManager: Loss was due to >> >> java.lang.ClassNotFoundException: >> >> org.apache.spark.rdd.RDD$$anonfun$saveAsTextFile$1 [duplicate 17] >> >> 14/06/06 09:05:12 INFO scheduler.DAGScheduler: Failed to run >> >> saveAsTextFile >> >> at Hello.scala:123 >> >> 14/06/06 09:05:12 INFO scheduler.TaskSchedulerImpl: Cancelling stage 0 >> >> [error] (run-main-0) org.apache.spark.SparkException: Job aborted due >> >> to >> >> stage failure: Task 0.0:3 failed 4 times, most recent failure: >> >> Exception >> >> failure in TID 142 on host sparky-s1.c.quick-heaven-560.internal: >> >> java.lang.ClassNotFoundException: >> >> org.apache.spark.rdd.RDD$$anonfun$saveAsTextFile$1 >> >> [error] java.net.URLClassLoader$1.run(URLClassLoader.java:366) >> >> [error] java.net.URLClassLoader$1.run(URLClassLoader.java:355) >> >> [error] java.security.AccessController.doPrivileged(Native >> >> Method) >> >> [error] >> >> java.net.URLClassLoader.findClass(URLClassLoader.java:354) >> >> [error] java.lang.ClassLoader.loadClass(ClassLoader.java:425) >> >> [error] java.lang.ClassLoader.loadClass(ClassLoader.java:358) >> >> [error] java.lang.Class.forName0(Native Method) >> >> [error] java.lang.Class.forName(Class.java:270) >> >> [error] >> >> >> >> >> >> org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:60) >> >> [error] >> >> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1612) >> >> [error] >> >> java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1517) >> >> [error] >> >> >> >> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1771) >> >> [error] >> >> java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) >> >> [error] >> >> >> >> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990) >> >> [error] >> >> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915) >> >> [error] >> >> >> >> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798) >> >> [error] >> >> java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) >> >> [error] >> >> java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) >> >> [error] >> >> >> >> >> >> org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:63) >> >> [error] >> >> >> >> org.apache.spark.scheduler.ResultTask$.deserializeInfo(ResultTask.scala:61) >> >> [error] >> >> >> >> org.apache.spark.scheduler.ResultTask.readExternal(ResultTask.scala:141) >> >> [error] >> >> java.io.ObjectInputStream.readExternalData(ObjectInputStream.java:1837) >> >> [error] >> >> >> >> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) >> >> [error] >> >> java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) >> >> [error] >> >> java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) >> >> [error] >> >> >> >> >> >> org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:63) >> >> [error] >> >> >> >> >> >> org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:85) >> >> [error] >> >> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:169) >> >> [error] >> >> >> >> >> >> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) >> >> [error] >> >> >> >> >> >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) >> >> [error] java.lang.Thread.run(Thread.java:744) >> >> >> >> Thanks for any help or guidance. >> >> >> >> >> >> >> >> >> >> >> >> -- >> >> View this message in context: >> >> >> >> http://apache-spark-user-list.1001560.n3.nabble.com/Strange-problem-with-saveAsTextFile-after-upgrade-Spark-0-9-0-1-0-0-tp6832p7122.html >> >> Sent from the Apache Spark User List mailing list archive at >> >> Nabble.com. >> >> > >