You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Francesco Bongiovanni <bo...@gmail.com> on 2014/02/04 11:20:51 UTC

spark 0.9.0 on top of Mesos error Akka Actor not found

Hi everyone, 

I installed the latest Spark release (0.9.0), on top of Mesos,  linked to my
HDFS 1.2.1 (sbt assembly success, make-distribution success), and when I try
to launch some ops from the spark-shell, I got the following error. I
configured my spark-env.sh and exported the correct env variables, but I am
stucked on this error. I have tried building Spark from the sources, from
the binaries with Hadoop1, cleaned my .ivy2 and .m2 caches, and the same
error arises...what am I missing ?

Here are my spark-env and the stderr from Mesos. 


=================SPARK-ENV.SH==============================
export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so
export
SPARK_EXECUTOR_URI=hdfs://10.10.0.141:9000/spark/spark-0.9.0-incubating.tgz
export MASTER=zk://10.10.0.141:2181/mesos
export SPARK_LOCAL_IP=10.10.0.141

if [ -z "$SPARK_MEM" ] ; then
  SPARK_MEM="15g"
fi


if [ -z "$SPARK_WORKER_MEMORY" ] ; then
  SPARK_WORKER_MEMORY="40g"
fi





===============STDERR=======================================
14/02/04 11:04:22 INFO MesosExecutorBackend: Using Spark's default log4j
profile: org/apache/spark/log4j-defaults.properties
14/02/04 11:04:22 INFO MesosExecutorBackend: Registered with Mesos as
executor ID 201402040838-2365590026-5050-31560-7
14/02/04 11:04:22 INFO Slf4jLogger: Slf4jLogger started
14/02/04 11:04:22 INFO Remoting: Starting remoting
14/02/04 11:04:22 INFO Remoting: Remoting started; listening on addresses
:[akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
14/02/04 11:04:22 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
14/02/04 11:04:23 INFO SparkEnv: Connecting to BlockManagerMaster:
akka.tcp://spark@localhost:7077/user/BlockManagerMaster
akka.actor.ActorNotFound: Actor not found for:
ActorSelection[Actor[akka.tcp://spark@localhost:7077/]/user/BlockManagerMaster]
	at
akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:66)
	at
akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:64)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at
akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1(BatchingExecutor.scala:67)
	at
akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:82)
	at
akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
	at
akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
	at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
	at akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala:58)
	at
akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatchedExecute(Future.scala:74)
	at akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.scala:110)
	at
akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execute(Future.scala:73)
	at
scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:269)
	at akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:512)
	at akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:545)
	at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:535)
	at
akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(RemoteActorRefProvider.scala:91)
	at akka.actor.ActorRef.tell(ActorRef.scala:125)
	at akka.dispatch.Mailboxes$$anon$1$$anon$2.enqueue(Mailboxes.scala:44)
	at akka.dispatch.QueueBasedMessageQueue$class.cleanUp(Mailbox.scala:438)
	at
akka.dispatch.UnboundedDequeBasedMailbox$MessageQueue.cleanUp(Mailbox.scala:650)
	at akka.dispatch.Mailbox.cleanUp(Mailbox.scala:309)
	at akka.dispatch.MessageDispatcher.unregister(AbstractDispatcher.scala:204)
	at akka.dispatch.MessageDispatcher.detach(AbstractDispatcher.scala:140)
	at
akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandling$$finishTerminate(FaultHandling.scala:203)
	at
akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala:163)
	at akka.actor.ActorCell.terminate(ActorCell.scala:338)
	at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:431)
	at akka.actor.ActorCell.systemInvoke(ActorCell.scala:447)
	at akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:262)
	at akka.dispatch.Mailbox.run(Mailbox.scala:218)
	at
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
	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)
Exception in thread "Thread-0" 



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Marek Wiewiorka <ma...@gmail.com>.
Hi Francesco,
I have exactly the same problem. I also use HDFS 1.2.1, Spark 0.9.0 and
Mesos 0.15.
If I run my application in coarse-grained(conf.set("spark.mesos.coarse",
"true"))
everything is ok but in fine-grained  mode the same exception is thrown:

akka.actor.ActorNotFound: Actor not found for:
ActorSelection[Actor[akka.tcp://spark@localhost:7077/]/user/
BlockManagerMaster]


I tried setting SPARK_LOCAL_IP env variable but it didn't help.

Regards,
Marek


2014-02-04 Francesco Bongiovanni <bo...@gmail.com>:

> Hi everyone,
>
> I installed the latest Spark release (0.9.0), on top of Mesos,  linked to
> my
> HDFS 1.2.1 (sbt assembly success, make-distribution success), and when I
> try
> to launch some ops from the spark-shell, I got the following error. I
> configured my spark-env.sh and exported the correct env variables, but I am
> stucked on this error. I have tried building Spark from the sources, from
> the binaries with Hadoop1, cleaned my .ivy2 and .m2 caches, and the same
> error arises...what am I missing ?
>
> Here are my spark-env and the stderr from Mesos.
>
>
> =================SPARK-ENV.SH==============================
> export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so
> export
> SPARK_EXECUTOR_URI=hdfs://
> 10.10.0.141:9000/spark/spark-0.9.0-incubating.tgz
> export MASTER=zk://10.10.0.141:2181/mesos
> export SPARK_LOCAL_IP=10.10.0.141
>
> if [ -z "$SPARK_MEM" ] ; then
>   SPARK_MEM="15g"
> fi
>
>
> if [ -z "$SPARK_WORKER_MEMORY" ] ; then
>   SPARK_WORKER_MEMORY="40g"
> fi
>
>
>
>
>
> ===============STDERR=======================================
> 14/02/04 11:04:22 INFO MesosExecutorBackend: Using Spark's default log4j
> profile: org/apache/spark/log4j-defaults.properties
> 14/02/04 11:04:22 INFO MesosExecutorBackend: Registered with Mesos as
> executor ID 201402040838-2365590026-5050-31560-7
> 14/02/04 11:04:22 INFO Slf4jLogger: Slf4jLogger started
> 14/02/04 11:04:22 INFO Remoting: Starting remoting
> 14/02/04 11:04:22 INFO Remoting: Remoting started; listening on addresses
> :[akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
> 14/02/04 11:04:22 INFO Remoting: Remoting now listens on addresses:
> [akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
> 14/02/04 11:04:23 INFO SparkEnv: Connecting to BlockManagerMaster:
> akka.tcp://spark@localhost:7077/user/BlockManagerMaster
> akka.actor.ActorNotFound: Actor not found for:
> ActorSelection[Actor[akka.tcp://spark@localhost
> :7077/]/user/BlockManagerMaster]
>         at
>
> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:66)
>         at
>
> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:64)
>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1(BatchingExecutor.scala:67)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:82)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>         at
> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>         at
> akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala:58)
>         at
>
> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatchedExecute(Future.scala:74)
>         at
> akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.scala:110)
>         at
>
> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execute(Future.scala:73)
>         at
> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
>         at
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
>         at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:269)
>         at akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:512)
>         at akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:545)
>         at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:535)
>         at
>
> akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(RemoteActorRefProvider.scala:91)
>         at akka.actor.ActorRef.tell(ActorRef.scala:125)
>         at
> akka.dispatch.Mailboxes$$anon$1$$anon$2.enqueue(Mailboxes.scala:44)
>         at
> akka.dispatch.QueueBasedMessageQueue$class.cleanUp(Mailbox.scala:438)
>         at
>
> akka.dispatch.UnboundedDequeBasedMailbox$MessageQueue.cleanUp(Mailbox.scala:650)
>         at akka.dispatch.Mailbox.cleanUp(Mailbox.scala:309)
>         at
> akka.dispatch.MessageDispatcher.unregister(AbstractDispatcher.scala:204)
>         at
> akka.dispatch.MessageDispatcher.detach(AbstractDispatcher.scala:140)
>         at
>
> akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandling$$finishTerminate(FaultHandling.scala:203)
>         at
> akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala:163)
>         at akka.actor.ActorCell.terminate(ActorCell.scala:338)
>         at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:431)
>         at akka.actor.ActorCell.systemInvoke(ActorCell.scala:447)
>         at
> akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:262)
>         at akka.dispatch.Mailbox.run(Mailbox.scala:218)
>         at
>
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
>         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)
> Exception in thread "Thread-0"
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Marek Wiewiorka <ma...@gmail.com>.
yes, indeed the URI should be fine, besides I don't use ZK and still
getting the same error...
M.


2014-02-04 Mark Hamstra <ma...@clearstorydata.com>:

> Nope, sorry -- looks like that particular issue has been fixed so that
> your URI should be fine.
>
>
> On Tue, Feb 4, 2014 at 2:33 AM, Mark Hamstra <ma...@clearstorydata.com>wrote:
>
>> export MASTER=mesos://zk://10.10.0.141:2181/mesos
>>
>>
>> On Tue, Feb 4, 2014 at 2:20 AM, Francesco Bongiovanni <
>> bongiovanni@gmail.com> wrote:
>>
>>> Hi everyone,
>>>
>>> I installed the latest Spark release (0.9.0), on top of Mesos,  linked
>>> to my
>>> HDFS 1.2.1 (sbt assembly success, make-distribution success), and when I
>>> try
>>> to launch some ops from the spark-shell, I got the following error. I
>>> configured my spark-env.sh and exported the correct env variables, but I
>>> am
>>> stucked on this error. I have tried building Spark from the sources, from
>>> the binaries with Hadoop1, cleaned my .ivy2 and .m2 caches, and the same
>>> error arises...what am I missing ?
>>>
>>> Here are my spark-env and the stderr from Mesos.
>>>
>>>
>>> =================SPARK-ENV.SH==============================
>>> export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so
>>> export
>>> SPARK_EXECUTOR_URI=hdfs://
>>> 10.10.0.141:9000/spark/spark-0.9.0-incubating.tgz
>>> export MASTER=zk://10.10.0.141:2181/mesos
>>> export SPARK_LOCAL_IP=10.10.0.141
>>>
>>> if [ -z "$SPARK_MEM" ] ; then
>>>   SPARK_MEM="15g"
>>> fi
>>>
>>>
>>> if [ -z "$SPARK_WORKER_MEMORY" ] ; then
>>>   SPARK_WORKER_MEMORY="40g"
>>> fi
>>>
>>>
>>>
>>>
>>>
>>> ===============STDERR=======================================
>>> 14/02/04 11:04:22 INFO MesosExecutorBackend: Using Spark's default log4j
>>> profile: org/apache/spark/log4j-defaults.properties
>>> 14/02/04 11:04:22 INFO MesosExecutorBackend: Registered with Mesos as
>>> executor ID 201402040838-2365590026-5050-31560-7
>>> 14/02/04 11:04:22 INFO Slf4jLogger: Slf4jLogger started
>>> 14/02/04 11:04:22 INFO Remoting: Starting remoting
>>> 14/02/04 11:04:22 INFO Remoting: Remoting started; listening on addresses
>>> :[akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
>>> 14/02/04 11:04:22 INFO Remoting: Remoting now listens on addresses:
>>> [akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
>>> 14/02/04 11:04:23 INFO SparkEnv: Connecting to BlockManagerMaster:
>>> akka.tcp://spark@localhost:7077/user/BlockManagerMaster
>>> akka.actor.ActorNotFound: Actor not found for:
>>> ActorSelection[Actor[akka.tcp://spark@localhost
>>> :7077/]/user/BlockManagerMaster]
>>>         at
>>>
>>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:66)
>>>         at
>>>
>>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:64)
>>>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>>>         at
>>>
>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1(BatchingExecutor.scala:67)
>>>         at
>>>
>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:82)
>>>         at
>>>
>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>>>         at
>>>
>>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>>>         at
>>> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>>>         at
>>> akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala:58)
>>>         at
>>>
>>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatchedExecute(Future.scala:74)
>>>         at
>>> akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.scala:110)
>>>         at
>>>
>>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execute(Future.scala:73)
>>>         at
>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
>>>         at
>>>
>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
>>>         at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:269)
>>>         at
>>> akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:512)
>>>         at
>>> akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:545)
>>>         at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:535)
>>>         at
>>>
>>> akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(RemoteActorRefProvider.scala:91)
>>>         at akka.actor.ActorRef.tell(ActorRef.scala:125)
>>>         at
>>> akka.dispatch.Mailboxes$$anon$1$$anon$2.enqueue(Mailboxes.scala:44)
>>>         at
>>> akka.dispatch.QueueBasedMessageQueue$class.cleanUp(Mailbox.scala:438)
>>>         at
>>>
>>> akka.dispatch.UnboundedDequeBasedMailbox$MessageQueue.cleanUp(Mailbox.scala:650)
>>>         at akka.dispatch.Mailbox.cleanUp(Mailbox.scala:309)
>>>         at
>>> akka.dispatch.MessageDispatcher.unregister(AbstractDispatcher.scala:204)
>>>         at
>>> akka.dispatch.MessageDispatcher.detach(AbstractDispatcher.scala:140)
>>>         at
>>>
>>> akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandling$$finishTerminate(FaultHandling.scala:203)
>>>         at
>>> akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala:163)
>>>         at akka.actor.ActorCell.terminate(ActorCell.scala:338)
>>>         at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:431)
>>>         at akka.actor.ActorCell.systemInvoke(ActorCell.scala:447)
>>>         at
>>> akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:262)
>>>         at akka.dispatch.Mailbox.run(Mailbox.scala:218)
>>>         at
>>>
>>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
>>>         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)
>>> Exception in thread "Thread-0"
>>>
>>>
>>>
>>> --
>>> View this message in context:
>>> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164.html
>>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>>
>>
>>
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Francesco Bongiovanni <bo...@gmail.com>.
hi, apparently it has been solved ! :) the old way of getting conf parameters
(using System.getProperties) was still hanging around in the
MesosSchedulerBackend.scala file. 

The fix is available here :
https://github.com/apache/incubator-spark/pull/568

and I confirm that it works ;) 





--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1377.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Kyle Ellrott <ke...@soe.ucsc.edu>.
I've been seeing the 'akka.actor.ActorNotFound' error as well when working
with 0.9 on Mesos. What fixes are working for people?

Kyle


On Fri, Feb 7, 2014 at 1:01 AM, Francesco Bongiovanni <bongiovanni@gmail.com
> wrote:

> So, still not working but playing around with some conf variables, I have a
> new error now...
>
> When I set the spark.driver.host and spark.driver.port variables in my
> spark-env.sh, I have the following error.
>
> The weird part is that when I go to the Spark homepage, and look at the
> environment, the spark.httpBroadcast.uri is set...:S
>
> I ll keep digging...:D
>
> 14/02/07 09:42:37 INFO MesosExecutorBackend: Using Spark's default log4j
> profile: org/apache/spark/log4j-defaults.properties
> 14/02/07 09:42:37 INFO MesosExecutorBackend: Registered with Mesos as
> executor ID 201402041543-2365590026-5050-17047-3
> 14/02/07 09:42:37 INFO Slf4jLogger: Slf4jLogger started
> 14/02/07 09:42:37 INFO Remoting: Starting remoting
> 14/02/07 09:42:38 INFO Remoting: Remoting started; listening on addresses
> :[akka.tcp://spark@srv-esx-isc04.crpgl.lu:40872]
> 14/02/07 09:42:38 INFO Remoting: Remoting now listens on addresses:
> [akka.tcp://spark@srv-esx-isc04.crpgl.lu:40872]
> 14/02/07 09:42:38 INFO SparkEnv: Connecting to BlockManagerMaster:
> akka.tcp://spark@srv-esx-isc01.crpgl.lu:38700/user/BlockManagerMaster
> 14/02/07 09:42:38 INFO DiskBlockManager: Created local directory at
> /tmp/spark-local-20140207094238-2712
> 14/02/07 09:42:38 INFO MemoryStore: MemoryStore started with capacity 8.6
> GB.
> 14/02/07 09:42:38 INFO ConnectionManager: Bound socket to port 47228 with
> id
> = ConnectionManagerId(srv-esx-isc04.crpgl.lu,47228)
> 14/02/07 09:42:38 INFO BlockManagerMaster: Trying to register BlockManager
> 14/02/07 09:42:38 INFO BlockManagerMaster: Registered BlockManager
> java.util.NoSuchElementException: spark.httpBroadcast.uri
>         at
> org.apache.spark.SparkConf$$anonfun$get$1.apply(SparkConf.scala:151)
>         at
> org.apache.spark.SparkConf$$anonfun$get$1.apply(SparkConf.scala:151)
>         at scala.collection.MapLike$class.getOrElse(MapLike.scala:128)
>         at scala.collection.AbstractMap.getOrElse(Map.scala:58)
>         at org.apache.spark.SparkConf.get(SparkConf.scala:151)
>         at
>
> org.apache.spark.broadcast.HttpBroadcast$.initialize(HttpBroadcast.scala:104)
>         at
>
> org.apache.spark.broadcast.HttpBroadcastFactory.initialize(HttpBroadcast.scala:70)
>         at
> org.apache.spark.broadcast.BroadcastManager.initialize(Broadcast.scala:81)
>         at
> org.apache.spark.broadcast.BroadcastManager.<init>(Broadcast.scala:68)
>         at org.apache.spark.SparkEnv$.create(SparkEnv.scala:176)
>         at org.apache.spark.executor.Executor.<init>(Executor.scala:110)
>         at
>
> org.apache.spark.executor.MesosExecutorBackend.registered(MesosExecutorBackend.scala:58)
> Exception in thread "Thread-0
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1303.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Francesco Bongiovanni <bo...@gmail.com>.
So, still not working but playing around with some conf variables, I have a
new error now...

When I set the spark.driver.host and spark.driver.port variables in my
spark-env.sh, I have the following error. 

The weird part is that when I go to the Spark homepage, and look at the
environment, the spark.httpBroadcast.uri is set...:S 

I ll keep digging...:D

14/02/07 09:42:37 INFO MesosExecutorBackend: Using Spark's default log4j
profile: org/apache/spark/log4j-defaults.properties
14/02/07 09:42:37 INFO MesosExecutorBackend: Registered with Mesos as
executor ID 201402041543-2365590026-5050-17047-3
14/02/07 09:42:37 INFO Slf4jLogger: Slf4jLogger started
14/02/07 09:42:37 INFO Remoting: Starting remoting
14/02/07 09:42:38 INFO Remoting: Remoting started; listening on addresses
:[akka.tcp://spark@srv-esx-isc04.crpgl.lu:40872]
14/02/07 09:42:38 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://spark@srv-esx-isc04.crpgl.lu:40872]
14/02/07 09:42:38 INFO SparkEnv: Connecting to BlockManagerMaster:
akka.tcp://spark@srv-esx-isc01.crpgl.lu:38700/user/BlockManagerMaster
14/02/07 09:42:38 INFO DiskBlockManager: Created local directory at
/tmp/spark-local-20140207094238-2712
14/02/07 09:42:38 INFO MemoryStore: MemoryStore started with capacity 8.6
GB.
14/02/07 09:42:38 INFO ConnectionManager: Bound socket to port 47228 with id
= ConnectionManagerId(srv-esx-isc04.crpgl.lu,47228)
14/02/07 09:42:38 INFO BlockManagerMaster: Trying to register BlockManager
14/02/07 09:42:38 INFO BlockManagerMaster: Registered BlockManager
java.util.NoSuchElementException: spark.httpBroadcast.uri
	at org.apache.spark.SparkConf$$anonfun$get$1.apply(SparkConf.scala:151)
	at org.apache.spark.SparkConf$$anonfun$get$1.apply(SparkConf.scala:151)
	at scala.collection.MapLike$class.getOrElse(MapLike.scala:128)
	at scala.collection.AbstractMap.getOrElse(Map.scala:58)
	at org.apache.spark.SparkConf.get(SparkConf.scala:151)
	at
org.apache.spark.broadcast.HttpBroadcast$.initialize(HttpBroadcast.scala:104)
	at
org.apache.spark.broadcast.HttpBroadcastFactory.initialize(HttpBroadcast.scala:70)
	at
org.apache.spark.broadcast.BroadcastManager.initialize(Broadcast.scala:81)
	at org.apache.spark.broadcast.BroadcastManager.<init>(Broadcast.scala:68)
	at org.apache.spark.SparkEnv$.create(SparkEnv.scala:176)
	at org.apache.spark.executor.Executor.<init>(Executor.scala:110)
	at
org.apache.spark.executor.MesosExecutorBackend.registered(MesosExecutorBackend.scala:58)
Exception in thread "Thread-0



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1303.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Xiaoli Li <li...@gmail.com>.
Same problem. It seems that workers cannot determine the driver's
hostname/ip and port correctly.


On Tue, Feb 4, 2014 at 4:10 PM, Alberto Miorin <sp...@ululi.it> wrote:

> Same error here. I've opened a ticket:
> https://spark-project.atlassian.net/browse/SPARK-1052
>
>
> On Tue, Feb 4, 2014 at 4:58 PM, Tobias Knaup <to...@knaup.me> wrote:
>
>>  I'm running into the same issue. Any help appreciated.
>>
>>
>> On Tuesday, February 4, 2014, Francesco Bongiovanni <
>> bongiovanni@gmail.com> wrote:
>>
>>> well, I also tried using spark.mesos.coarse=true and yeah it works...but
>>> with
>>> the finer-grained, nope, still the akka actor not found...I tried setting
>>> values in the /etc/hosts, resolv.conf, etc...nothing seems to work...I ll
>>> keep digging
>>>
>>>
>>>
>>> --
>>> View this message in context:
>>> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1187.html
>>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>>
>>
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Alberto Miorin <sp...@ululi.it>.
Same error here. I've opened a ticket:
https://spark-project.atlassian.net/browse/SPARK-1052


On Tue, Feb 4, 2014 at 4:58 PM, Tobias Knaup <to...@knaup.me> wrote:

>  I'm running into the same issue. Any help appreciated.
>
>
> On Tuesday, February 4, 2014, Francesco Bongiovanni <bo...@gmail.com>
> wrote:
>
>> well, I also tried using spark.mesos.coarse=true and yeah it works...but
>> with
>> the finer-grained, nope, still the akka actor not found...I tried setting
>> values in the /etc/hosts, resolv.conf, etc...nothing seems to work...I ll
>> keep digging
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1187.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Tobias Knaup <to...@knaup.me>.
 I'm running into the same issue. Any help appreciated.

On Tuesday, February 4, 2014, Francesco Bongiovanni <bo...@gmail.com>
wrote:

> well, I also tried using spark.mesos.coarse=true and yeah it works...but
> with
> the finer-grained, nope, still the akka actor not found...I tried setting
> values in the /etc/hosts, resolv.conf, etc...nothing seems to work...I ll
> keep digging
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1187.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Francesco Bongiovanni <bo...@gmail.com>.
well, I also tried using spark.mesos.coarse=true and yeah it works...but with
the finer-grained, nope, still the akka actor not found...I tried setting
values in the /etc/hosts, resolv.conf, etc...nothing seems to work...I ll
keep digging



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1187.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Francesco Bongiovanni <bo...@gmail.com>.
It gets weirder now. 

I tried not using Mesos, and run Spark on standalone mode and it works. 

In the process, I discovered something that might hint me...so, I basically
run the spark master and let some slaves connect to it, with the following
command 

./bin/spark-class org.apache.spark.deploy.worker.Worker
spark://srv-esx-isc01:7077

and it works fine, my slaves connect to the master. 

But when I try this 

./bin/spark-class org.apache.spark.deploy.worker.Worker
spark://10.10.0.141:7077

it does not work, and the IP address is correct...

I will try changing the /etc/hosts file on every node that is supposed to
run mesos and hard code the IPs cuz I am pretty sure the actor which is not
found is bound to a crappy address 

I ll keep u posted







--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164p1178.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Mark Hamstra <ma...@clearstorydata.com>.
Nope, sorry -- looks like that particular issue has been fixed so that your
URI should be fine.


On Tue, Feb 4, 2014 at 2:33 AM, Mark Hamstra <ma...@clearstorydata.com>wrote:

> export MASTER=mesos://zk://10.10.0.141:2181/mesos
>
>
> On Tue, Feb 4, 2014 at 2:20 AM, Francesco Bongiovanni <
> bongiovanni@gmail.com> wrote:
>
>> Hi everyone,
>>
>> I installed the latest Spark release (0.9.0), on top of Mesos,  linked to
>> my
>> HDFS 1.2.1 (sbt assembly success, make-distribution success), and when I
>> try
>> to launch some ops from the spark-shell, I got the following error. I
>> configured my spark-env.sh and exported the correct env variables, but I
>> am
>> stucked on this error. I have tried building Spark from the sources, from
>> the binaries with Hadoop1, cleaned my .ivy2 and .m2 caches, and the same
>> error arises...what am I missing ?
>>
>> Here are my spark-env and the stderr from Mesos.
>>
>>
>> =================SPARK-ENV.SH==============================
>> export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so
>> export
>> SPARK_EXECUTOR_URI=hdfs://
>> 10.10.0.141:9000/spark/spark-0.9.0-incubating.tgz
>> export MASTER=zk://10.10.0.141:2181/mesos
>> export SPARK_LOCAL_IP=10.10.0.141
>>
>> if [ -z "$SPARK_MEM" ] ; then
>>   SPARK_MEM="15g"
>> fi
>>
>>
>> if [ -z "$SPARK_WORKER_MEMORY" ] ; then
>>   SPARK_WORKER_MEMORY="40g"
>> fi
>>
>>
>>
>>
>>
>> ===============STDERR=======================================
>> 14/02/04 11:04:22 INFO MesosExecutorBackend: Using Spark's default log4j
>> profile: org/apache/spark/log4j-defaults.properties
>> 14/02/04 11:04:22 INFO MesosExecutorBackend: Registered with Mesos as
>> executor ID 201402040838-2365590026-5050-31560-7
>> 14/02/04 11:04:22 INFO Slf4jLogger: Slf4jLogger started
>> 14/02/04 11:04:22 INFO Remoting: Starting remoting
>> 14/02/04 11:04:22 INFO Remoting: Remoting started; listening on addresses
>> :[akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
>> 14/02/04 11:04:22 INFO Remoting: Remoting now listens on addresses:
>> [akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
>> 14/02/04 11:04:23 INFO SparkEnv: Connecting to BlockManagerMaster:
>> akka.tcp://spark@localhost:7077/user/BlockManagerMaster
>> akka.actor.ActorNotFound: Actor not found for:
>> ActorSelection[Actor[akka.tcp://spark@localhost
>> :7077/]/user/BlockManagerMaster]
>>         at
>>
>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:66)
>>         at
>>
>> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:64)
>>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>>         at
>>
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1(BatchingExecutor.scala:67)
>>         at
>>
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:82)
>>         at
>>
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>>         at
>>
>> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>>         at
>> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>>         at
>> akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala:58)
>>         at
>>
>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatchedExecute(Future.scala:74)
>>         at
>> akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.scala:110)
>>         at
>>
>> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execute(Future.scala:73)
>>         at
>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
>>         at
>>
>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
>>         at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:269)
>>         at akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:512)
>>         at akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:545)
>>         at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:535)
>>         at
>>
>> akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(RemoteActorRefProvider.scala:91)
>>         at akka.actor.ActorRef.tell(ActorRef.scala:125)
>>         at
>> akka.dispatch.Mailboxes$$anon$1$$anon$2.enqueue(Mailboxes.scala:44)
>>         at
>> akka.dispatch.QueueBasedMessageQueue$class.cleanUp(Mailbox.scala:438)
>>         at
>>
>> akka.dispatch.UnboundedDequeBasedMailbox$MessageQueue.cleanUp(Mailbox.scala:650)
>>         at akka.dispatch.Mailbox.cleanUp(Mailbox.scala:309)
>>         at
>> akka.dispatch.MessageDispatcher.unregister(AbstractDispatcher.scala:204)
>>         at
>> akka.dispatch.MessageDispatcher.detach(AbstractDispatcher.scala:140)
>>         at
>>
>> akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandling$$finishTerminate(FaultHandling.scala:203)
>>         at
>> akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala:163)
>>         at akka.actor.ActorCell.terminate(ActorCell.scala:338)
>>         at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:431)
>>         at akka.actor.ActorCell.systemInvoke(ActorCell.scala:447)
>>         at
>> akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:262)
>>         at akka.dispatch.Mailbox.run(Mailbox.scala:218)
>>         at
>>
>> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
>>         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)
>> Exception in thread "Thread-0"
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>
>
>

Re: spark 0.9.0 on top of Mesos error Akka Actor not found

Posted by Mark Hamstra <ma...@clearstorydata.com>.
export MASTER=mesos://zk://10.10.0.141:2181/mesos


On Tue, Feb 4, 2014 at 2:20 AM, Francesco Bongiovanni <bongiovanni@gmail.com
> wrote:

> Hi everyone,
>
> I installed the latest Spark release (0.9.0), on top of Mesos,  linked to
> my
> HDFS 1.2.1 (sbt assembly success, make-distribution success), and when I
> try
> to launch some ops from the spark-shell, I got the following error. I
> configured my spark-env.sh and exported the correct env variables, but I am
> stucked on this error. I have tried building Spark from the sources, from
> the binaries with Hadoop1, cleaned my .ivy2 and .m2 caches, and the same
> error arises...what am I missing ?
>
> Here are my spark-env and the stderr from Mesos.
>
>
> =================SPARK-ENV.SH==============================
> export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so
> export
> SPARK_EXECUTOR_URI=hdfs://
> 10.10.0.141:9000/spark/spark-0.9.0-incubating.tgz
> export MASTER=zk://10.10.0.141:2181/mesos
> export SPARK_LOCAL_IP=10.10.0.141
>
> if [ -z "$SPARK_MEM" ] ; then
>   SPARK_MEM="15g"
> fi
>
>
> if [ -z "$SPARK_WORKER_MEMORY" ] ; then
>   SPARK_WORKER_MEMORY="40g"
> fi
>
>
>
>
>
> ===============STDERR=======================================
> 14/02/04 11:04:22 INFO MesosExecutorBackend: Using Spark's default log4j
> profile: org/apache/spark/log4j-defaults.properties
> 14/02/04 11:04:22 INFO MesosExecutorBackend: Registered with Mesos as
> executor ID 201402040838-2365590026-5050-31560-7
> 14/02/04 11:04:22 INFO Slf4jLogger: Slf4jLogger started
> 14/02/04 11:04:22 INFO Remoting: Starting remoting
> 14/02/04 11:04:22 INFO Remoting: Remoting started; listening on addresses
> :[akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
> 14/02/04 11:04:22 INFO Remoting: Remoting now listens on addresses:
> [akka.tcp://spark@srv-esx-isc04.crpgl.lu:56046]
> 14/02/04 11:04:23 INFO SparkEnv: Connecting to BlockManagerMaster:
> akka.tcp://spark@localhost:7077/user/BlockManagerMaster
> akka.actor.ActorNotFound: Actor not found for:
> ActorSelection[Actor[akka.tcp://spark@localhost
> :7077/]/user/BlockManagerMaster]
>         at
>
> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:66)
>         at
>
> akka.actor.ActorSelection$$anonfun$resolveOne$1.apply(ActorSelection.scala:64)
>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.processBatch$1(BatchingExecutor.scala:67)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:82)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>         at
>
> akka.dispatch.BatchingExecutor$Batch$$anonfun$run$1.apply(BatchingExecutor.scala:59)
>         at
> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>         at
> akka.dispatch.BatchingExecutor$Batch.run(BatchingExecutor.scala:58)
>         at
>
> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.unbatchedExecute(Future.scala:74)
>         at
> akka.dispatch.BatchingExecutor$class.execute(BatchingExecutor.scala:110)
>         at
>
> akka.dispatch.ExecutionContexts$sameThreadExecutionContext$.execute(Future.scala:73)
>         at
> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
>         at
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
>         at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:269)
>         at akka.actor.EmptyLocalActorRef.specialHandle(ActorRef.scala:512)
>         at akka.actor.DeadLetterActorRef.specialHandle(ActorRef.scala:545)
>         at akka.actor.DeadLetterActorRef.$bang(ActorRef.scala:535)
>         at
>
> akka.remote.RemoteActorRefProvider$RemoteDeadLetterActorRef.$bang(RemoteActorRefProvider.scala:91)
>         at akka.actor.ActorRef.tell(ActorRef.scala:125)
>         at
> akka.dispatch.Mailboxes$$anon$1$$anon$2.enqueue(Mailboxes.scala:44)
>         at
> akka.dispatch.QueueBasedMessageQueue$class.cleanUp(Mailbox.scala:438)
>         at
>
> akka.dispatch.UnboundedDequeBasedMailbox$MessageQueue.cleanUp(Mailbox.scala:650)
>         at akka.dispatch.Mailbox.cleanUp(Mailbox.scala:309)
>         at
> akka.dispatch.MessageDispatcher.unregister(AbstractDispatcher.scala:204)
>         at
> akka.dispatch.MessageDispatcher.detach(AbstractDispatcher.scala:140)
>         at
>
> akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandling$$finishTerminate(FaultHandling.scala:203)
>         at
> akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala:163)
>         at akka.actor.ActorCell.terminate(ActorCell.scala:338)
>         at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:431)
>         at akka.actor.ActorCell.systemInvoke(ActorCell.scala:447)
>         at
> akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:262)
>         at akka.dispatch.Mailbox.run(Mailbox.scala:218)
>         at
>
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
>         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)
> Exception in thread "Thread-0"
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/spark-0-9-0-on-top-of-Mesos-error-Akka-Actor-not-found-tp1164.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>