spark-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Koert Kuipers <ko...@tresata.com>
Subject Re: spark 0.8
Date Fri, 18 Oct 2013 18:02:06 GMT
i checked out the v0.8.0-incubating tag again, changed the settings to
build against correct version of hadoop for our cluster, ran sbt-assembly,
build tarball, installed it on cluster, restarted spark... same errors


On Fri, Oct 18, 2013 at 12:49 PM, Koert Kuipers <koert@tresata.com> wrote:

> at this point i feel like it must be some sort of version mismatch? i am
> gonna check the spark build that i deployed on the cluster
>
>
> On Fri, Oct 18, 2013 at 12:46 PM, Koert Kuipers <koert@tresata.com> wrote:
>
>> name := "Simple Project"
>>
>> version := "1.0"
>>
>> scalaVersion := "2.9.3"
>>
>> libraryDependencies += "org.apache.spark" %% "spark-core" %
>> "0.8.0-incubating"
>>
>> resolvers += "Akka Repository" at "http://repo.akka.io/releases/"
>>
>> resolvers += "Cloudera Repository" at "
>> https://repository.cloudera.com/artifactory/cloudera-repos/"
>>
>> libraryDependencies += "org.apache.hadoop" % "hadoop-client" %
>> "2.0.0-mr1-cdh4.3.0"
>>
>>
>>
>>
>> On Fri, Oct 18, 2013 at 12:34 PM, Matei Zaharia <matei.zaharia@gmail.com>wrote:
>>
>>> Can you post the build.sbt for your program? It needs to include
>>> hadoop-client for CDH4.3, and that should *not* be listed as provided.
>>>
>>> Matei
>>>
>>> On Oct 18, 2013, at 8:23 AM, Koert Kuipers <koert@tresata.com> wrote:
>>>
>>> ok this has nothing to do with hadoop access. even a simple program that
>>> uses sc.parallelize blows up in this way.
>>>
>>> so spark-shell works well on the same machine i launch this from.
>>>
>>> if i launch a simple program without using kryo for serializer and
>>> closure serialize i get a different error. see below.
>>> at this point it seems to me i have some issue with task serialization???
>>>
>>>
>>>
>>> 13/10/18 11:20:37 INFO StandaloneExecutorBackend: Got assigned task 0
>>> 13/10/18 11:20:37 INFO StandaloneExecutorBackend: Got assigned task 1
>>> 13/10/18 11:20:37 INFO Executor: Running task ID 1
>>> 13/10/18 11:20:37 INFO Executor: Running task ID 0
>>> 13/10/18 11:20:37 INFO Executor: Fetching
>>> http://192.168.3.171:41629/jars/simple-project_2.9.3-1.0.jar with
>>> timestamp 1382109635095
>>> 13/10/18 11:20:37 INFO Utils: Fetching
>>> http://192.168.3.171:41629/jars/simple-project_2.9.3-1.0.jar to
>>> /tmp/fetchFileTemp378181753997570700.tmp
>>> 13/10/18 11:20:37 INFO Executor: Adding
>>> file:/var/lib/spark/app-20131018112035-0014/1/./simple-project_2.9.3-1.0.jar
>>> to class loader
>>> 13/10/18 11:20:37 INFO Executor: caught throwable with stacktrace
>>> java.io.StreamCorruptedException: invalid type code: 00
>>>     at
>>> java.io.ObjectInputStream$BlockDataInputStream.readBlockHeader(ObjectInputStream.java:2467)
>>>     at
>>> java.io.ObjectInputStream$BlockDataInputStream.refill(ObjectInputStream.java:2502)
>>>     at
>>> java.io.ObjectInputStream$BlockDataInputStream.read(ObjectInputStream.java:2661)
>>>     at
>>> java.io.ObjectInputStream$BlockDataInputStream.read(ObjectInputStream.java:2583)
>>>     at java.io.DataInputStream.readFully(DataInputStream.java:178)
>>>     at java.io.DataInputStream.readLong(DataInputStream.java:399)
>>>     at
>>> java.io.ObjectInputStream$BlockDataInputStream.readLong(ObjectInputStream.java:2803)
>>>     at java.io.ObjectInputStream.readLong(ObjectInputStream.java:958)
>>>     at
>>> org.apache.spark.rdd.ParallelCollectionPartition.readObject(ParallelCollectionRDD.scala:72)
>>>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>     at
>>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>>>     at
>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>>     at java.lang.reflect.Method.invoke(Method.java:597)
>>>     at
>>> java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:969)
>>>     at
>>> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1852)
>>>     at
>>> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1756)
>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1326)
>>>     at java.io.ObjectInputStream.readObject(ObjectInputStream.java:348)
>>>     at
>>> org.apache.spark.scheduler.ResultTask.readExternal(ResultTask.scala:135)
>>>     at
>>> java.io.ObjectInputStream.readExternalData(ObjectInputStream.java:1795)
>>>     at
>>> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1754)
>>>     at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1326)
>>>     at java.io.ObjectInputStream.readObject(ObjectInputStream.java:348)
>>>     at
>>> org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:39)
>>>     at
>>> org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:61)
>>>     at
>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:153)
>>>     at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
>>>     at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>>>     at java.lang.Thread.run(Thread.java:662)
>>>
>>>
>>>
>>> On Fri, Oct 18, 2013 at 10:59 AM, Koert Kuipers <koert@tresata.com>wrote:
>>>
>>>> i created a tiny sbt project as described here:
>>>> apache.org/docs/latest/quick-start.html#a-standalone-app-in-scala<http://spark.incubator.apache.org/docs/latest/quick-start.html#a-standalone-app-in-scala>
>>>>
>>>> it has the correct dependencies: spark-core and the correct
>>>> hadoop-client for my platform. i tried both the generic spark-core
>>>> dependency and spark-core dependency compiled against my platform. it runs
>>>> fine in local mode, but when i switch to the cluster i still always get the
>>>> following exceptions on tasks:
>>>>
>>>> 13/10/18 10:25:53 ERROR Executor: Uncaught exception in thread
>>>> Thread[pool-5-thread-1,5,main]
>>>>
>>>> java.lang.NullPointerException
>>>>     at
>>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:187)
>>>>     at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
>>>>     at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>>>>     at java.lang.Thread.run(Thread.java:662)
>>>>
>>>> after adding some additional debugging to Executor i see the cause is
>>>> this:
>>>> 13/10/18 10:54:47 INFO Executor: caught throwable with stacktrace
>>>> java.lang.NullPointerException
>>>>     at
>>>> org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$2.apply(Executor.scala:155)
>>>>     at
>>>> org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$2.apply(Executor.scala:155)
>>>>     at org.apache.spark.Logging$class.logInfo(Logging.scala:48)
>>>>     at org.apache.spark.executor.Executor.logInfo(Executor.scala:36)
>>>>     at
>>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:155)
>>>>
>>>>     at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
>>>>     at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>>>>     at java.lang.Thread.run(Thread.java:662)
>>>>
>>>> so it seems the offending line is:
>>>> logInfo("Its epoch is " + task.epoch)
>>>>
>>>> i am guessing accessing epoch on the task is throwing the NPE. any
>>>> ideas?
>>>>
>>>>
>>>>
>>>> On Thu, Oct 17, 2013 at 8:12 PM, Koert Kuipers <koert@tresata.com>wrote:
>>>>
>>>>> sorry one more related question:
>>>>> i compile against a spark build for hadoop 1.0.4, but the actual
>>>>> installed version of spark is build against cdh4.3.0-mr1. this also used
to
>>>>> work, and i prefer to do this so i compile against a generic spark build.
>>>>> could this be the issue?
>>>>>
>>>>>
>>>>> On Thu, Oct 17, 2013 at 8:06 PM, Koert Kuipers <koert@tresata.com>wrote:
>>>>>
>>>>>> i have my spark and hadoop related dependencies as "provided" for
my
>>>>>> spark job. this used to work with previous versions. are these now
supposed
>>>>>> to be compile/runtime/default dependencies?
>>>>>>
>>>>>>
>>>>>> On Thu, Oct 17, 2013 at 8:04 PM, Koert Kuipers <koert@tresata.com>wrote:
>>>>>>
>>>>>>> yes i did that and i can see the correct jars sitting in lib_managed
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Oct 17, 2013 at 7:56 PM, Matei Zaharia <
>>>>>>> matei.zaharia@gmail.com> wrote:
>>>>>>>
>>>>>>>> Koert, did you link your Spark job to the right version of
HDFS as
>>>>>>>> well? In Spark 0.8, you have to add a Maven dependency on
"hadoop-client"
>>>>>>>> for your version of Hadoop. See
>>>>>>>> http://spark.incubator.apache.org/docs/latest/quick-start.html#a-standalone-app-in-scala
for
>>>>>>>> example.
>>>>>>>>
>>>>>>>> Matei
>>>>>>>>
>>>>>>>> On Oct 17, 2013, at 4:38 PM, Koert Kuipers <koert@tresata.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> i got the job a little further along by also setting this:
>>>>>>>> System.setProperty("spark.closure.serializer",
>>>>>>>> "org.apache.spark.serializer.KryoSerializer")
>>>>>>>>
>>>>>>>> not sure why i need to... but anyhow, now my workers start
and then
>>>>>>>> they blow up on this:
>>>>>>>>
>>>>>>>> 13/10/17 19:22:57 ERROR Executor: Uncaught exception in thread
>>>>>>>> Thread[pool-5-thread-1,5,main]
>>>>>>>> java.lang.NullPointerException
>>>>>>>>     at
>>>>>>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:187)
>>>>>>>>     at
>>>>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
>>>>>>>>     at
>>>>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>>>>>>>>     at java.lang.Thread.run(Thread.java:662)
>>>>>>>>
>>>>>>>>
>>>>>>>> which is:
>>>>>>>>  val metrics = attemptedTask.flatMap(t => t.metrics)
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Oct 17, 2013 at 7:30 PM, dachuan <hdc1112@gmail.com>
wrote:
>>>>>>>>
>>>>>>>>> thanks, Mark.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Oct 17, 2013 at 6:36 PM, Mark Hamstra <
>>>>>>>>> mark@clearstorydata.com> wrote:
>>>>>>>>>
>>>>>>>>>> SNAPSHOTs are not fixed versions, but are floating
names
>>>>>>>>>> associated with whatever is the most recent code.
 So, Spark 0.8.0 is the
>>>>>>>>>> current released version of Spark, which is exactly
the same today as it
>>>>>>>>>> was yesterday, and will be the same thing forever.
 Spark 0.8.1-SNAPSHOT is
>>>>>>>>>> whatever is currently in branch-0.8.  It changes
every time new code is
>>>>>>>>>> committed to that branch (which should be just bug
fixes and the few
>>>>>>>>>> additional features that we wanted to get into 0.8.0,
but that didn't quite
>>>>>>>>>> make it.)  Not too long from now there will be a
release of Spark 0.8.1, at
>>>>>>>>>> which time the SNAPSHOT will got to 0.8.2 and 0.8.1
will be forever frozen.
>>>>>>>>>>  Meanwhile, the wild new development is taking place
on the master branch,
>>>>>>>>>> and whatever is currently in that branch becomes
0.9.0-SNAPSHOT.  This
>>>>>>>>>> could be quite different from day to day, and there
are no guarantees that
>>>>>>>>>> things won't be broken in 0.9.0-SNAPSHOT.  Several
months from now there
>>>>>>>>>> will be a release of Spark 0.9.0 (unless the decision
is made to bump the
>>>>>>>>>> version to 1.0.0), at which point the SNAPSHOT goes
to 0.9.1 and the whole
>>>>>>>>>> process advances to the next phase of development.
>>>>>>>>>>
>>>>>>>>>> The short answer is that releases are stable, SNAPSHOTs
are not,
>>>>>>>>>> and SNAPSHOTs that aren't on maintenance branches
can break things.  You
>>>>>>>>>> make your choice of which to use and pay the consequences.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Oct 17, 2013 at 3:18 PM, dachuan <hdc1112@gmail.com>wrote:
>>>>>>>>>>
>>>>>>>>>>> yeah, I mean 0.9.0-SNAPSHOT. I use git clone
and that's what I
>>>>>>>>>>> got.. what's the difference? I mean SNAPSHOT
and non-SNAPSHOT.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Oct 17, 2013 at 6:15 PM, Mark Hamstra
<
>>>>>>>>>>> mark@clearstorydata.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Of course, you mean 0.9.0-SNAPSHOT.  There
is no Spark 0.9.0,
>>>>>>>>>>>> and won't be for several months.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Oct 17, 2013 at 3:11 PM, dachuan
<hdc1112@gmail.com>wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I'm sorry if this doesn't answer your
question directly, but I
>>>>>>>>>>>>> have tried spark 0.9.0 and hdfs 1.0.4
just now, it works..
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Oct 17, 2013 at 6:05 PM, Koert
Kuipers <
>>>>>>>>>>>>> koert@tresata.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> after upgrading from spark 0.7 to
spark 0.8 i can no longer
>>>>>>>>>>>>>> access any files on HDFS.
>>>>>>>>>>>>>>  i see the error below. any ideas?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> i am running spark standalone on
a cluster that also has
>>>>>>>>>>>>>> CDH4.3.0 and rebuild spark accordingly.
the jars in lib_managed look good
>>>>>>>>>>>>>> to me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> i noticed similar errors in the mailing
list but found no
>>>>>>>>>>>>>> suggested solutions.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> thanks! koert
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/17 17:43:23 ERROR Executor:
Exception in task ID 0
>>>>>>>>>>>>>> java.io.EOFException
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream$BlockDataInputStream.readFully(ObjectInputStream.java:2703)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readFully(ObjectInputStream.java:1008)
>>>>>>>>>>>>>> 	at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:68)
>>>>>>>>>>>>>> 	at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:106)
>>>>>>>>>>>>>> 	at org.apache.hadoop.io.UTF8.readChars(UTF8.java:258)
>>>>>>>>>>>>>> 	at org.apache.hadoop.io.UTF8.readString(UTF8.java:250)
>>>>>>>>>>>>>> 	at org.apache.hadoop.mapred.FileSplit.readFields(FileSplit.java:87)
>>>>>>>>>>>>>> 	at org.apache.hadoop.io.ObjectWritable.readObject(ObjectWritable.java:280)
>>>>>>>>>>>>>> 	at org.apache.hadoop.io.ObjectWritable.readFields(ObjectWritable.java:75)
>>>>>>>>>>>>>> 	at org.apache.spark.SerializableWritable.readObject(SerializableWritable.scala:39)
>>>>>>>>>>>>>> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native
Method)
>>>>>>>>>>>>>> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>>>>>>>>>>>>>> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>>>>>>>>>>>>> 	at java.lang.reflect.Method.invoke(Method.java:597)
>>>>>>>>>>>>>> 	at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:969)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1852)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1756)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1326)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1950)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1874)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1756)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1326)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:348)
>>>>>>>>>>>>>> 	at org.apache.spark.scheduler.ResultTask.readExternal(ResultTask.scala:135)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readExternalData(ObjectInputStream.java:1795)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1754)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1326)
>>>>>>>>>>>>>> 	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:348)
>>>>>>>>>>>>>> 	at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:39)
>>>>>>>>>>>>>> 	at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:61)
>>>>>>>>>>>>>> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:153)
>>>>>>>>>>>>>> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
>>>>>>>>>>>>>> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>>>>>>>>>>>>>> 	at java.lang.Thread.run(Thread.java:662)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --
>>>>>>>>>>>>> Dachuan Huang
>>>>>>>>>>>>> Cellphone: 614-390-7234
>>>>>>>>>>>>> 2015 Neil Avenue
>>>>>>>>>>>>> Ohio State University
>>>>>>>>>>>>> Columbus, Ohio
>>>>>>>>>>>>> U.S.A.
>>>>>>>>>>>>> 43210
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> Dachuan Huang
>>>>>>>>>>> Cellphone: 614-390-7234
>>>>>>>>>>> 2015 Neil Avenue
>>>>>>>>>>> Ohio State University
>>>>>>>>>>> Columbus, Ohio
>>>>>>>>>>> U.S.A.
>>>>>>>>>>> 43210
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> Dachuan Huang
>>>>>>>>> Cellphone: 614-390-7234
>>>>>>>>> 2015 Neil Avenue
>>>>>>>>> Ohio State University
>>>>>>>>> Columbus, Ohio
>>>>>>>>> U.S.A.
>>>>>>>>> 43210
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>>
>>
>

Mime
View raw message