You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Imran Rashid <im...@quantifind.com> on 2013/10/29 20:59:15 UTC

executor failures w/ scala 2.10

We've been testing out the 2.10 branch of spark, and we're running into
some issues were akka disconnects from the executors after a while.  We ran
some simple tests first, and all was well, so we started upgrading our
whole codebase to 2.10.  Everything seemed to be working, but then we
noticed that when we run long jobs, and then things start failing.


The first suspicious thing is that we get akka warnings about undeliverable
messages sent to deadLetters:

22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO
akka.actor.LocalActorRef - Message
[akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
Actor[akka://spark/deadLetters] to
Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
was not delivered. [4] dead letters encountered. This logging can be turned
off or adjusted with configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.

2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO
akka.actor.LocalActorRef - Message
[akka.remote.transport.AssociationHandle$Disassociated] from
Actor[akka://spark/deadLetters] to
Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
was not delivered. [5] dead letters encountered. This logging can be turned
off or adjusted with configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.



Generally within a few seconds after the first such message, there are a
bunch more, and then the executor is marked as failed, and a new one is
started:

2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
akka.actor.LocalActorRef - Message
[akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
Actor[akka://spark/deadLetters] to
Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10] dead
letters encountered, no more dead letters will be logged. This logging can
be turned off or adjusted with configuration settings
'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.

2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO
org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
app-20131029110000-0000/1 is now FAILED (Command exited with code 1)

2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO
org.apache.spark.deploy.client.Client$ClientActor - Executor added:
app-20131029110000-0000/2 on
worker-20131029105824-dhd2.quantifind.com-51544 (dhd2.quantifind.com:51544)
with 24 cores

2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR
akka.remote.EndpointWriter - AssociationError [akka.tcp://
spark@ddd0.quantifind.com:43068] -> [akka.tcp://
sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed with
[akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
akka.remote.EndpointAssociationException: Association failed with
[akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
Caused by:
akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
Connection refused: dhd2.quantifind.com/10.10.5.64:45794]



Looking in the logs of the failed executor, there are some similar messages
about undeliverable messages, but I don't see any reason:

13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943

13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.actor.FSM$Timer]
from Actor[akka://sparkExecutor/deadLetters] to
Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead
letters encountered. This logging can be turned off or adjusted with
configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.

13/10/29 11:03:53 INFO actor.LocalActorRef: Message
[akka.remote.transport.AssociationHandle$Disassociated] from
Actor[akka://sparkExecutor/deadLetters] to
Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead
letters encountered. This logging can be turned off or adjusted with
configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.

13/10/29 11:03:53 INFO actor.LocalActorRef: Message
[akka.remote.transport.AssociationHandle$Disassociated] from
Actor[akka://sparkExecutor/deadLetters] to
Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead
letters encountered. This logging can be turned off or adjusted with
configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.

13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
terminated or disconnected! Shutting down.

13/10/29 11:03:53 INFO actor.LocalActorRef: Message
[akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
Actor[akka://sparkExecutor/deadLetters] to
Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead
letters encountered. This logging can be turned off or adjusted with
configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.


After this happens, spark does launch a new executor successfully, and
continue the job.  Sometimes, the job just continues happily and there
aren't any other problems.  However, that executor may have to run a bunch
of steps to re-compute some cached RDDs -- and during that time, another
executor may crash similarly, and then we end up in a never ending loop, of
one executor crashing, then trying to reload data, while the others sit
around.

I have no idea what is triggering this behavior -- there isn't any
particular point in the job that it regularly occurs at.  Certain steps
seem more prone to this, but there isn't any step which regularly causes
the problem.  In a long pipeline of steps, though, that loop becomes very
likely.  I don't think its a timeout issue -- the initial failing executors
can be actively completing stages just seconds before this failure
happens.  We did try adjusting some of the spark / akka timeouts:

    -Dspark.storage.blockManagerHeartBeatMs=300000
    -Dspark.akka.frameSize=150
    -Dspark.akka.timeout=120
    -Dspark.akka.askTimeout=30
    -Dspark.akka.logLifecycleEvents=true

but those settings didn't seem to help the problem at all.  I figure it
must be some configuration with the new version of akka that we're missing,
but we haven't found anything.  Any ideas?

our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures
occur on the tip of the scala-2.10 branch (5429d62d)

thanks,
Imran

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
I see, that’s quite interesting. For problem 2, I think the issue is that Akka 2.0.5 *always* kept TCP connections open between nodes, so these messages didn’t get lost. It looks like Akka 2.2 occasionally disconnects them and loses messages. If this is the case, and this behavior can’t be disabled with a flag, then it’s a problem for other parts of the code too. Most of our code assumes that messages will make it through unless the destination node dies, which is what you’d usually hope for TCP.

Matei

On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:

> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
> 
> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
> 
> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
> 
> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
> 
> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
> 
> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
> 
> imran
> 
> 
> 
> On Thu, Oct 31, 2013 at 1:17 PM, Imran Rashid <im...@quantifind.com> wrote:
> Hi Prashant,
> 
> thanks for looking into this.  I don't have any answers yet, but just wanted to send you an update.  I finally figured out how to get all the akka logging turned on, so I'm looking at those for more info.  One thing immediately jumped out at me -- the Disassociation is actually immediatley followed by an Association!  so maybe I came to the wrong conclusion of our test of ignoring the DisassociatedEvent.  I'm going to try it again -- hopefully w/ the logging on, I can find out more about what is going on.  I might ask on akka list for help w/ what to look for.  also this thread makes me think that it really should just re-associate:
> https://groups.google.com/forum/#!searchin/akka-user/Disassociated/akka-user/SajwwbyTriQ/8oxjbZtawxoJ
> 
> also, I've noticed that actually these failures are *not* more likely the longer the job runs (or not that much, anyway).  when I reorder our jobs, I can get the failures pretty regularly within 10 minutes.  It seems to be more of a function of what the jobs do.  I'll also see if I can produce a simple example which exhibits the same problem.
> 
> thanks!
> 
> Imran
> 
> 
> 
> On Thu, Oct 31, 2013 at 1:37 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Hi Imran,
> 
> So now my jobs are running for more than 18 hours and there is not a single executor failure, I can keep it running for more but I don't think that is going to help. So either you have to give me a way to reproduce the issue (Which would be great !! ) or you have to diagnose for yourself what exactly is causing it, a disconnection usually result in Disassociation event on which we are terminating the executor. In a normal operation this usually means that driver has finished and those log messages are totally fine for that matter. I am not sure why you would not see such a thing in 0.8 release and in scala 2.10, can't say anything until I can see what is happening. 
> 
> 
> 
> On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Can you show us the sample job ? do you do sc.stop at the end or System.exit ? Try sc.stop too..
> 
> 
> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com> wrote:
> yeah, just causes them to hang.
> 
> the first "deadLetters" message shows up about the same time.  Oddly, after it first happens, I keep getting some results trickling in from those executors.  (maybe they were just queued up on the driver already, I dunno.)  but then it just hangs.  the stage has a few more tasks to be run, but the executors are just idle, they're not running anything.
> 
> I'm gonna try manually listening for more Association events listed here & logging them
> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
> 
> imran
> 
> 
> 
> 
> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <sc...@gmail.com> wrote:
> I am guessing something wrong with using Dissociation event then.
> 
> Try applying something on the lines of this patch. This might cause the executors to hang so be prepared for that. 
> 
> diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> index 4e8052a..1ec5d19 100644
> --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>          executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
>        }
>  
> -    case DisassociatedEvent(_, _, _) =>
> -      logError("Driver terminated or disconnected! Shutting down.")
> +    case Terminated(actor) =>
> +      logError("Driver terminated Shutting down.")
>        System.exit(1)
> +
> +    // case DisassociatedEvent(_, _, _) =>
> +    //   logError("Driver terminated or disconnected! Shutting down.")
> +    //   System.exit(1)
>    }
>  
>    override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) {
> diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> index b6f0ec9..9955484 100644
> --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
>        case Terminated(actor) =>
>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
>  
> -      case DisassociatedEvent(_, remoteAddress, _) =>
> -        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected"))
> +      // case DisassociatedEvent(_, remoteAddress, _) =>
> +      //   addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected"))
>  
>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>          addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown"))
> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
>      // Remove a disconnected slave from the cluster
>      def removeExecutor(executorId: String, reason: String) {
>        if (executorActor.contains(executorId)) {
> -        logInfo("Executor " + executorId + " disconnected, so removing it")
> +        logInfo("Executor " + executorId + " disconnected, so removing it, reason:" + reason)
>          val numCores = freeCores(executorId)
>          actorToExecutorId -= executorActor(executorId)
>          addressToExecutorId -= executorAddress(executorId)
> 
> 
> 
> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com> wrote:
> ok, so I applied a few patches
> 
> https://github.com/quantifind/incubator-spark/pull/1/files
> 
> and ran it again, with these options:
> 
> -Dspark.akka.stdout-loglevel=DEBUG \
>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>   -Dspark.akkaExtra.akka.actor.debug.receive=on \  -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>   -Dspark.akkaExtra.akka.log-config-on-start=on
> 
> On the driver, I see:
> 
> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141] was not delivered. [1] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO  org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1 disconnected, so removing it, reason:remote Akka client disconnected
> 
> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on dhd2.quantifind.com: remote Akka client disconnected
> 
> 
> on the worker, stderr:
> 
> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
> 
> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver terminated or disconnected! Shutting down.Disassociated [akka.tcp://sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://spark@ddd0.quantifind.com:36730]
> 
> and unfortunately, all those akka debug options give me *no* useful info in the worker stdout:
> 
> Starting akka system "sparkExecutor" using config:
> 
>       akka.daemonic = on
>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>       akka.stdout-loglevel = "DEBUG"
>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>       akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>       akka.remote.netty.tcp.port = 0
>       akka.remote.netty.tcp.connection-timeout = 60 s
>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>       akka.remote.netty.tcp.execution-pool-size = 4
>       akka.actor.default-dispatcher.throughput = 15
>       akka.remote.log-remote-lifecycle-events = off
>                        akka.remote.log-sent-messages = on
> akka.remote.log-received-messages = on
> akka.logLevel = DEBUG
> akka.actor.debug.autoreceive = on
> akka.actor.debug.lifecycle = on
> akka.actor.debug.receive = on
> akka.log-config-on-start = on
> akka.remote.quarantine-systems-for = off
> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream] StandardOutLogger started
> [DEBUG] [10/30/2013 08:40:30.438] [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
> [DEBUG] [10/30/2013 08:40:30.446] [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user] started (akka.actor.LocalActorRefProvider$Guardian@72608760)
> [DEBUG] [10/30/2013 08:40:30.447] [sparkExecutor-akka.actor.default-dispatcher-4] [akka://sparkExecutor/system] started (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
> [DEBUG] [10/30/2013 08:40:30.454] [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now supervising Actor[akka://sparkExecutor/user]
> [DEBUG] [10/30/2013 08:40:30.454] [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now supervising Actor[akka://sparkExecutor/system]
> [DEBUG] [10/30/2013 08:40:30.468] [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user] now monitoring Actor[akka://sparkExecutor/system]
> [DEBUG] [10/30/2013 08:40:30.468] [sparkExecutor-akka.actor.default-dispatcher-4] [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
> [DEBUG] [10/30/2013 08:40:30.476] [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/system/log1-Slf4jLogger] started (akka.event.slf4j.Slf4jLogger@24988707)
> [DEBUG] [10/30/2013 08:40:30.477] [sparkExecutor-akka.actor.default-dispatcher-4] [akka://sparkExecutor/system] now supervising Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
> 
> (followed by similar mesages for the "spark" system)
> 
> I dunno if this means much more to you, but it seems to me that for some reason the executor decides to disconnect from the master -- unfortunately we don't know why.  I think my logging configuration is not getting applied correctly, or "log-sent-messages" & "log-received-messages" don't do what I think they do ... something conflicting must be turing that logging off.  There are a zillion different remoting settings:
> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
> 
> I feel like I really need to get those messages on why it disconnected to know which ones to play with.  Any ideas for config changes to see those messages?
> 
> thanks
> 
> 
> 
> 
> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Can you apply this patch too and check the logs of Driver and worker.
> 
> diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> index b6f0ec9..ad0ebf7 100644
> --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
>      // Remove a disconnected slave from the cluster
>      def removeExecutor(executorId: String, reason: String) {
>        if (executorActor.contains(executorId)) {
> -        logInfo("Executor " + executorId + " disconnected, so removing it")
> +        logInfo("Executor " + executorId + " disconnected, so removing it, reason:" + reason)
>          val numCores = freeCores(executorId)
>          actorToExecutorId -= executorActor(executorId)
>          addressToExecutorId -= executorAddress(executorId)
> 
> 
> 
> 
> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com> wrote:
> I just realized something about the failing stages -- they generally occur in steps like this:
> 
> rdd.mapPartitions{itr =>
>   val myCounters = initializeSomeDataStructure()
>   itr.foreach{
>     //update myCounter in here
>     ...
>   }
>   
>   myCounters.iterator.map{ 
>     //some other transformation here ...
>   }
> }
> 
> that is, as a partition is processed, nothing gets output, we just accumulate some values.  Only at the end of the partition do we output some accumulated values.
> 
> These stages don't always fail, and generally they do succeed after the executor has died and a new one has started -- so I'm pretty confident its not a problem w/ the code.  But maybe we need to add something like a periodic heartbeat in this kind of operation?
> 
> 
> 
> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com> wrote:
> I'm gonna try turning on more akka debugging msgs as described at
> http://akka.io/faq/
> and
> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
> 
> unfortunately that will require a patch to spark, but hopefully that will give us more info to go on ...
> 
> 
> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com> wrote:
> I have things running (from scala 2.10 branch) for over 3-4 hours now without a problem and my jobs write data about the same as you suggested. My cluster size is 7 nodes and not *congested* for memory. I going to leave jobs running all night long. Meanwhile I had encourage you to try to spot the problem such that it is reproducible that can help a ton in fixing the issue. 
> 
> Thanks for testing and reporting your experience. I still feel there is something else wrong !. About tolerance for network connection timeouts, setting those properties should work, but I am afraid about Disassociation Event though. I will have to check this is indeed hard to reproduce bug if it is, I mean how do I simulate network delays ?
> 
> 
> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com> wrote:
> This is a spark-standalone setup (not mesos), on our own cluster.
> 
> At first I thought it must be some temporary network problem too -- but the times between receiving task completion events from an executor and declaring it failed are really small, so I didn't think that could possibly be it.  Plus we tried increasing various akka timeouts, but that didn't help.  Or maybe there are some other spark / akka properities we should be setting?  It certainly should be resilient to such a temporary network issue, if that is the problem.
> 
> btw, I think I've noticed this happens most often during ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB total for the entire stage).
> 
> thanks
> 
> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Are you using mesos ? I admit to have not properly tested things on mesos though. 
> 
> 
> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Those log messages are new to the Akka 2.2 and are usually seen when a node is disassociated with other by either a network failure or even clean shutdown. This suggests some network issue to me, are you running on EC2 ? It might be a temporary thing in that case. 
> 
> I had like to have more details on the long jobs though, how long ? 
> 
> 
> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com> wrote:
> We've been testing out the 2.10 branch of spark, and we're running into some issues were akka disconnects from the executors after a while.  We ran some simple tests first, and all was well, so we started upgrading our whole codebase to 2.10.  Everything seemed to be working, but then we noticed that when we run long jobs, and then things start failing.
> 
> 
> The first suspicious thing is that we get akka warnings about undeliverable messages sent to deadLetters:
> 
> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700] was not delivered. [4] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700] was not delivered. [5] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 
> 
> Generally within a few seconds after the first such message, there are a bunch more, and then the executor is marked as failed, and a new one is started:
> 
> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10] dead letters encountered, no more dead letters will be logged. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated: app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
> 
> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added: app-20131029110000-0000/2 on worker-20131029105824-dhd2.quantifind.com-51544 (dhd2.quantifind.com:51544) with 24 cores
> 
> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068] -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
> akka.remote.EndpointAssociationException: Association failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
> Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
> 
> 
> 
> Looking in the logs of the failed executor, there are some similar messages about undeliverable messages, but I don't see any reason:
> 
> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver terminated or disconnected! Shutting down.
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 
> After this happens, spark does launch a new executor successfully, and continue the job.  Sometimes, the job just continues happily and there aren't any other problems.  However, that executor may have to run a bunch of steps to re-compute some cached RDDs -- and during that time, another executor may crash similarly, and then we end up in a never ending loop, of one executor crashing, then trying to reload data, while the others sit around.
> 
> I have no idea what is triggering this behavior -- there isn't any particular point in the job that it regularly occurs at.  Certain steps seem more prone to this, but there isn't any step which regularly causes the problem.  In a long pipeline of steps, though, that loop becomes very likely.  I don't think its a timeout issue -- the initial failing executors can be actively completing stages just seconds before this failure happens.  We did try adjusting some of the spark / akka timeouts:
> 
>     -Dspark.storage.blockManagerHeartBeatMs=300000
>     -Dspark.akka.frameSize=150
>     -Dspark.akka.timeout=120
>     -Dspark.akka.askTimeout=30
>     -Dspark.akka.logLifecycleEvents=true
> 
> but those settings didn't seem to help the problem at all.  I figure it must be some configuration with the new version of akka that we're missing, but we haven't found anything.  Any ideas?
> 
> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures occur on the tip of the scala-2.10 branch (5429d62d)
> 
> thanks,
> Imran
> 
> 
> 
> -- 
> s
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> -- 
> s
> 
> 


Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
We can set timeouts high enough ! same as connection timeout that we
already set.


On Wed, Nov 13, 2013 at 11:37 PM, Matei Zaharia <ma...@gmail.com>wrote:

> Hey Prashant, do messages still get lost while we’re dissociated? Or can
> you set the timeouts high enough to proven that?
>
> Matei
>
> On Nov 13, 2013, at 12:39 AM, Prashant Sharma <sc...@gmail.com>
> wrote:
>
> We may no longer need to track disassociation and IMHO use the *improved*
> feature in akka 2.2.x called remote death watch. Which lets us acknowledge
> a remote death both in case of a natural demise and accidental deaths. This
> was not the case with remote death watch in previous akka releases. Please
> take a closer look at the patch at
> https://github.com/apache/incubator-spark/pull/163 and let us know.
>
> This patch does not make disassociation disappear, they are added to akka
> as such but gives us sufficient knobs to tune things as to when they occur.
> Don't forget to tune those extra properties apart from other timeouts.
>
>
>
>
>
> On Sat, Nov 2, 2013 at 11:08 AM, Matei Zaharia <ma...@gmail.com>wrote:
>
>> Prashant, the problem seems to be that messages sent while we’re
>> disassociated are lost. I think we’d have to just prevent disassociation
>> altogether, or replace all remote actor refs with the reliable proxies
>> (which sounds painful).
>>
>> Matei
>>
>> On Nov 1, 2013, at 7:53 PM, Prashant Sharma <sc...@gmail.com> wrote:
>>
>> Hey Matei and Imran,
>>
>> I think may be we can solve the problem without downgrading to 2.1.0 may
>> by capturing dissociation and then setting a timeout if it associates again
>> we keep moving else we shutdown the executor. This timeout can ofcourse be
>> configurable.
>>
>> Thoughts ?
>>
>>
>> On Sat, Nov 2, 2013 at 3:29 AM, Matei Zaharia <ma...@gmail.com>wrote:
>>
>>> Hey Imran,
>>>
>>> Good to know that Akka 2.1 handles this — that at least will give us a
>>> start.
>>>
>>> In the old code, executors certainly did get flagged as “down”
>>> occasionally, but that was due to a timeout we controlled (we keep sending
>>> heartbeats back and forth to track them). The timeout used to be smaller
>>> and usually the reason to exceed it was GC. However, if Akka 2.2 can
>>> sometimes drop the connections itself, this is a problem and we either have
>>> to use the reliable proxies for everything or see if we can configure it
>>> otherwise. Anyway, we’ll definitely look into it.
>>>
>>> Matei
>>>
>>> On Nov 1, 2013, at 1:09 PM, Imran Rashid <im...@quantifind.com> wrote:
>>>
>>> I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm
>>> going to run my tests a few more times , but I'd really have expected to
>>> see a failure by now w/ the 2.2.3 version.
>>>
>>> I'll submit a patch shortly (need to fix some compile errors in
>>> streaming still).
>>>
>>> Matei -- I think I realize now that when you were talking about the
>>> expectation of a tcp connection staying alive, you were explaining why this
>>> is *not* a bug in the current release.  You wouldn't end up in a situation
>>> where the executor thinks it finished the task, but the driver doesn't know
>>> about it, b/c if the connection dies, the executor wil get restarted.  That
>>> makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of
>>> things change.  I was probably wrong about seeing that problem in previous
>>> releases -- it was just a vague recollection, which fit my current
>>> theories, so I jumped to conclusions.
>>>
>>> thanks everyone
>>>
>>>
>>>
>>> On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> thanks everyone for all of the input.
>>>>
>>>> Matei: makes a lot more sense with your explanation of spark's expected
>>>> behavior of tcp, I can see why this makes sense now.  But, to show my total
>>>> ignorance here, I'm wondering that when the connection does break, are you
>>>> sure all of your messages that you thought you sent before the break were
>>>> received?  I'm guessing that you don't.  Which is fine, if the response to
>>>> that is to have the executor just die completely, and restart.  that was
>>>> the behavior I was initially observing with the code on the 2.10 branch,
>>>> where the executor handles a DisassociatedEvent explicitly, and dies.
>>>>
>>>> But -- is that the behavior we want?  do we want it to be robust to tcp
>>>> connections breaking, without having to completely restart the executor?
>>>> you might say that dying & restarting will lead to correct behavior, even
>>>> if its inefficient.  But sometimes, I've seen restarts so frequently that
>>>> no progress is made.
>>>>
>>>> I don't see why this changed w/ the different versions of akka -- I
>>>> don't see any relevant configuration settings that would change how
>>>> "strongly" tcp tries to keep the connection alive, but I may be missing
>>>> something.  But it does seem like the netty configuration options have
>>>> changed completely between the two versions:
>>>>
>>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
>>>> vs
>>>> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
>>>>
>>>> btw, akka 2.1.0 also has been built for scala 2.10:
>>>>
>>>> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
>>>> and its netty configuration is closer to 2.0.5:
>>>> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
>>>>
>>>> perhaps someone more knowledge then me about netty & tcp can look
>>>> through the changes and decide what the right changes are.
>>>>
>>>> Prashant said:
>>>> >Before we conclude something about reliable messaging, I want you to
>>>> for once consider other possibilities like >actual network reconnection and
>>>> may be a GC pause ? Try connecting something like jconsole (or alike ) and
>>>> >see what happens on the driver and executor.
>>>> >
>>>> >My doubt are since we are using standalone mode where even master and
>>>> worker are also actors then if we see >a weird behaviour on the executor
>>>> and driver then Why not on master and worker too ? They should also break
>>>> >away from each other. For this reason, I am doubting our conclusions and
>>>> may be if we narrow down the >problem first before we conclude something.
>>>> It is a regression in akka 2.2.3 it uses more memory than it used to >be in
>>>> 2.1.x.
>>>> >See https://github.com/akka/akka/issues/1810
>>>>
>>>>
>>>> Well, there could easily be the same problem with dropped connections
>>>> between master & worker -- they just communicate so little, it doesn't
>>>> really matter.  The odds that a message gets dropped between them is very
>>>> low, only because there are barely any messages.
>>>>
>>>> I completely agree that the problem could be because of a contention,
>>>> or gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores
>>>> available on each box, and 90g out of 125g memory.  I've looked at gc a
>>>> little with jstat, and I did see some gc pauses but nothing ridiculous.
>>>>
>>>> But, I think the question remains.  Suppose it is gc pauses, etc. that
>>>> cause the disassociation events; what do we do to fix it?  How can we
>>>> diagnose the problem, and figure out which of the configuration variables
>>>> to tune?  clearly, there *will be* long gc pauses, and the networking layer
>>>> needs to be able to deal with them.
>>>>
>>>> still I understand your desire to see if that might be the cause of the
>>>> problem in this particular case, so I will dig a little more.
>>>>
>>>>
>>>> (btw, should I move this thread to the dev list now?  it is getting
>>>> into the nitty-gritty of implementation ...)
>>>>
>>>> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com>wrote:
>>>>
>>>>> Yes, so far they’ve been built on that assumption — not that Akka
>>>>> would *guarantee* delivery in that as soon as the send() call returns you
>>>>> know it’s delivered, but that Akka would act the same way as a TCP socket,
>>>>> allowing you to send a stream of messages in order and hear when the
>>>>> connection breaks. Maybe that isn’t what they want to provide, but I'd find
>>>>> it weird, because it’s very easy to write a server with this property.
>>>>>
>>>>> Matei
>>>>>
>>>>> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <
>>>>> sri.rams85@gmail.com> wrote:
>>>>>
>>>>> Sorry if I my understanding is wrong. May be, for this particular case
>>>>> it might be something to do with the load/network, but, in general, are you
>>>>> saying that, we build these communication channels(block manager
>>>>> communication, task events communication, etc) assuming akka would take
>>>>> care of it? I somehow feel that, it's being overly optimistic. Correct me
>>>>> if I am wrong.
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <
>>>>> matei.zaharia@gmail.com> wrote:
>>>>>
>>>>>> It’s true that Akka’s delivery guarantees are in general
>>>>>> at-most-once, but if you look at the text there it says that they differ by
>>>>>> transport. In the previous version, I’m quite sure that except maybe in
>>>>>> very rare circumstances or cases where we had a bug, Akka’s remote layer
>>>>>> always kept connections up between each pair of hosts. So the guarantee was
>>>>>> that as long as you haven’t received a “disconnected” event, your messages
>>>>>> are being delivered, though of course when you do receive that event you
>>>>>> don’t know which messages have really made it through unless you acked
>>>>>> them. But that didn’t matter for our use case — from our point of view an
>>>>>> executor was either up or down.
>>>>>>
>>>>>> For this reason I still think it should be possible to configure Akka
>>>>>> to do the same on 2.2. Most likely some timeouts just got lower. With large
>>>>>> heaps you can easily get a GC pause of 60 seconds, so these timeouts should
>>>>>> be in the minutes.
>>>>>>
>>>>>> If for some reason this isn’t the case, then we have a bigger problem
>>>>>> — there are *lots* of messages beyond task-finished that need to be sent
>>>>>> reliably, including things like block manager events (a block was added /
>>>>>> removed on this node) and commands to tell the block manager to drop data.
>>>>>> It would be silly to implement acks at the application level for all these.
>>>>>> But I doubt this is the case. Prashant’s observation that the standalone
>>>>>> cluster manager stayed up is a further sign that this might be due to GC.
>>>>>>
>>>>>> Matei
>>>>>>
>>>>>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <
>>>>>> sri.rams85@gmail.com> wrote:
>>>>>>
>>>>>> Hi Imran,
>>>>>> Just to add, we've noticed dis-associations in a couple projects that
>>>>>> we built(using akka 2.2.x not spark). We went to some details to find out
>>>>>> what was happening. As Matei, suggested, Akka keeps the TCP connection open
>>>>>> and uses that to talk to peers. We noticed that in our case, initially, we
>>>>>> were seeing dis-associations generally at the end of keep-alive duration.
>>>>>> So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe
>>>>>> gets sent to inform the peer on the other side that the connection is still
>>>>>> alive/valid. For some reason, the probe dint renew the keep-alive
>>>>>> connection and we saw a lot of dis-associations during that time. Later, we
>>>>>> realized this was not a pattern either. This thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains the full history of our discussions with the Akka team. It's still
>>>>>> open and unclear as to what was causing it for our case.
>>>>>> We tried tweaking various settings of akka(wrt heartbeats, failure
>>>>>> detector, even plugged-in our own failure detector with no effect).
>>>>>>
>>>>>> Imran - Just to clarify your point on message delivery - akka's
>>>>>> message delivery policy is at-most-once. However, there's no guarantee for
>>>>>> a message to be delivered to a peer. The documentation clearly explains
>>>>>> that.
>>>>>> http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
>>>>>> the responsibility of the application developer to handle cases where
>>>>>> message is suspected to be not have been delivered.
>>>>>>
>>>>>> I hope this helps.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>
>>>>>>>
>>>>>>> unfortunately that change wasn't the silver bullet I was hoping
>>>>>>> for.  Even with
>>>>>>> 1) ignoring DisassociatedEvent
>>>>>>> 2) executor uses ReliableProxy to send messages back to driver
>>>>>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>>>>>
>>>>>>>
>>>>>>> there is a lot of weird behavior.  First, there are a few
>>>>>>> DisassociatedEvents, but some that are followed by AssociatedEvents, so
>>>>>>> that seems ok.  But sometimes the re-associations are immediately followed
>>>>>>> by this:
>>>>>>>
>>>>>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
>>>>>>> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
>>>>>>> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
>>>>>>> akka.tcp://spark@<driver>:41321] [
>>>>>>> akka.remote.InvalidAssociation: Invalid address:
>>>>>>> akka.tcp://spark@<driver>:41321
>>>>>>> Caused by:
>>>>>>> akka.remote.transport.Transport$InvalidAssociationException: The remote
>>>>>>> system has quarantined this system. No further associations to the remote
>>>>>>> system are possible until this system is restarted.
>>>>>>> ]
>>>>>>>
>>>>>>> On the driver, there are messages like:
>>>>>>>
>>>>>>> [INFO] [10/31/2013 18:51:07.838]
>>>>>>> [spark-akka.actor.default-dispatcher-3] [Remoting] Address [
>>>>>>> akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all
>>>>>>> messages to this address will be delivered to dead letters.
>>>>>>> [WARN] [10/31/2013 18:51:10.845]
>>>>>>> [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher]
>>>>>>> Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>>>>>
>>>>>>>
>>>>>>> and when the driver does decide that the executor has been
>>>>>>> terminated, it removes the executor, but doesn't start another one.
>>>>>>>
>>>>>>> there are a ton of messages also about messages to the block manager
>>>>>>> master ... I'm wondering if there are other parts of the system that need
>>>>>>> to use a reliable proxy (or some sort of acknowledgement).
>>>>>>>
>>>>>>> I really don't think this was working properly even w/ previous
>>>>>>> versions of spark / akka.  I'm still learning about akka, but I think you
>>>>>>> always need an ack to be confident w/ remote communicate.  Perhaps the old
>>>>>>> version of akka just had more robust defaults or something, but I bet it
>>>>>>> could still have the same problems.  Even before, I have seen the driver
>>>>>>> thinking there were running tasks, but nothing happening on any executor --
>>>>>>> it was just rare enough (and hard to reproduce) that I never bothered
>>>>>>> looking into it more.
>>>>>>>
>>>>>>> I will keep digging ...
>>>>>>>
>>>>>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <
>>>>>>> matei.zaharia@gmail.com> wrote:
>>>>>>>
>>>>>>>> BTW the problem might be the Akka failure detector settings that
>>>>>>>> seem new in 2.2:
>>>>>>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>>>>>>
>>>>>>>> Their timeouts seem pretty aggressive by default — around 10
>>>>>>>> seconds. This can easily be too little if you have large garbage
>>>>>>>> collections. We should make sure they are higher than our own node failure
>>>>>>>> detection timeouts.
>>>>>>>>
>>>>>>>> Matei
>>>>>>>>
>>>>>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> pretty sure I found the problem -- two problems actually.  And I
>>>>>>>> think one of them has been a general lurking problem w/ spark for a while.
>>>>>>>>
>>>>>>>> 1)  we should ignore disassociation events, as you suggested
>>>>>>>> earlier.  They seem to just indicate a temporary problem, and can generally
>>>>>>>> be ignored.  I've found that they're regularly followed by
>>>>>>>> AssociatedEvents, and it seems communication really works fine at that
>>>>>>>> point.
>>>>>>>>
>>>>>>>> 2) Task finished messages get lost.  When this message gets sent,
>>>>>>>> we dont' know it actually gets there:
>>>>>>>>
>>>>>>>>
>>>>>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>>>>>
>>>>>>>> (this is so incredible, I feel I must be overlooking something --
>>>>>>>> but there is no ack somewhere else that I'm overlooking, is there??)  So,
>>>>>>>> after the patch, spark wasn't hanging b/c of the unhandled
>>>>>>>> DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished
>>>>>>>> messages that never get received by the driver.  So the driver is waiting
>>>>>>>> for some tasks to finish, but the executors think they are all done.
>>>>>>>>
>>>>>>>> I'm gonna add the reliable proxy pattern for this particular
>>>>>>>> interaction and see if its fixes the problem
>>>>>>>>
>>>>>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>>>>>
>>>>>>>> imran
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> It's just about how deep your longing is!
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> It's just about how deep your longing is!
>>>>>
>>>>>
>>>
>>>
>>
>>
>> --
>> s
>>
>>
>>
>
>
> --
> s
>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
Hey Prashant, do messages still get lost while we’re dissociated? Or can you set the timeouts high enough to proven that?

Matei

On Nov 13, 2013, at 12:39 AM, Prashant Sharma <sc...@gmail.com> wrote:

> We may no longer need to track disassociation and IMHO use the *improved* feature in akka 2.2.x called remote death watch. Which lets us acknowledge a remote death both in case of a natural demise and accidental deaths. This was not the case with remote death watch in previous akka releases. Please take a closer look at the patch at https://github.com/apache/incubator-spark/pull/163 and let us know.
> 
> This patch does not make disassociation disappear, they are added to akka as such but gives us sufficient knobs to tune things as to when they occur. Don't forget to tune those extra properties apart from other timeouts. 
> 
> 
> 
> 
> 
> On Sat, Nov 2, 2013 at 11:08 AM, Matei Zaharia <ma...@gmail.com> wrote:
> Prashant, the problem seems to be that messages sent while we’re disassociated are lost. I think we’d have to just prevent disassociation altogether, or replace all remote actor refs with the reliable proxies (which sounds painful).
> 
> Matei
> 
> On Nov 1, 2013, at 7:53 PM, Prashant Sharma <sc...@gmail.com> wrote:
> 
>> Hey Matei and Imran,
>> 
>> I think may be we can solve the problem without downgrading to 2.1.0 may by capturing dissociation and then setting a timeout if it associates again we keep moving else we shutdown the executor. This timeout can ofcourse be configurable. 
>> 
>> Thoughts ?
>> 
>> 
>> On Sat, Nov 2, 2013 at 3:29 AM, Matei Zaharia <ma...@gmail.com> wrote:
>> Hey Imran,
>> 
>> Good to know that Akka 2.1 handles this — that at least will give us a start.
>> 
>> In the old code, executors certainly did get flagged as “down” occasionally, but that was due to a timeout we controlled (we keep sending heartbeats back and forth to track them). The timeout used to be smaller and usually the reason to exceed it was GC. However, if Akka 2.2 can sometimes drop the connections itself, this is a problem and we either have to use the reliable proxies for everything or see if we can configure it otherwise. Anyway, we’ll definitely look into it.
>> 
>> Matei
>> 
>> On Nov 1, 2013, at 1:09 PM, Imran Rashid <im...@quantifind.com> wrote:
>> 
>>> I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm going to run my tests a few more times , but I'd really have expected to see a failure by now w/ the 2.2.3 version.
>>> 
>>> I'll submit a patch shortly (need to fix some compile errors in streaming still).
>>> 
>>> Matei -- I think I realize now that when you were talking about the expectation of a tcp connection staying alive, you were explaining why this is *not* a bug in the current release.  You wouldn't end up in a situation where the executor thinks it finished the task, but the driver doesn't know about it, b/c if the connection dies, the executor wil get restarted.  That makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of things change.  I was probably wrong about seeing that problem in previous releases -- it was just a vague recollection, which fit my current theories, so I jumped to conclusions.
>>> 
>>> thanks everyone
>>> 
>>> 
>>> 
>>> On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com> wrote:
>>> thanks everyone for all of the input.
>>> 
>>> Matei: makes a lot more sense with your explanation of spark's expected behavior of tcp, I can see why this makes sense now.  But, to show my total ignorance here, I'm wondering that when the connection does break, are you sure all of your messages that you thought you sent before the break were received?  I'm guessing that you don't.  Which is fine, if the response to that is to have the executor just die completely, and restart.  that was the behavior I was initially observing with the code on the 2.10 branch, where the executor handles a DisassociatedEvent explicitly, and dies.
>>> 
>>> But -- is that the behavior we want?  do we want it to be robust to tcp connections breaking, without having to completely restart the executor?  you might say that dying & restarting will lead to correct behavior, even if its inefficient.  But sometimes, I've seen restarts so frequently that no progress is made.
>>> 
>>> I don't see why this changed w/ the different versions of akka -- I don't see any relevant configuration settings that would change how "strongly" tcp tries to keep the connection alive, but I may be missing something.  But it does seem like the netty configuration options have changed completely between the two versions:
>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
>>> vs
>>> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
>>> 
>>> btw, akka 2.1.0 also has been built for scala 2.10:
>>> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
>>> and its netty configuration is closer to 2.0.5:
>>> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
>>> 
>>> perhaps someone more knowledge then me about netty & tcp can look through the changes and decide what the right changes are.
>>> 
>>> Prashant said:
>>> >Before we conclude something about reliable messaging, I want you to for once consider other possibilities like >actual network reconnection and may be a GC pause ? Try connecting something like jconsole (or alike ) and >see what happens on the driver and executor.
>>> >
>>> >My doubt are since we are using standalone mode where even master and worker are also actors then if we see >a weird behaviour on the executor and driver then Why not on master and worker too ? They should also break >away from each other. For this reason, I am doubting our conclusions and may be if we narrow down the >problem first before we conclude something. It is a regression in akka 2.2.3 it uses more memory than it used to >be in 2.1.x.  
>>> >See https://github.com/akka/akka/issues/1810 
>>> 
>>> 
>>> Well, there could easily be the same problem with dropped connections between master & worker -- they just communicate so little, it doesn't really matter.  The odds that a message gets dropped between them is very low, only because there are barely any messages.
>>> 
>>> I completely agree that the problem could be because of a contention, or gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available on each box, and 90g out of 125g memory.  I've looked at gc a little with jstat, and I did see some gc pauses but nothing ridiculous.
>>> 
>>> But, I think the question remains.  Suppose it is gc pauses, etc. that cause the disassociation events; what do we do to fix it?  How can we diagnose the problem, and figure out which of the configuration variables to tune?  clearly, there *will be* long gc pauses, and the networking layer needs to be able to deal with them.
>>> 
>>> still I understand your desire to see if that might be the cause of the problem in this particular case, so I will dig a little more.
>>> 
>>> 
>>> (btw, should I move this thread to the dev list now?  it is getting into the nitty-gritty of implementation ...)
>>> 
>>> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com> wrote:
>>> Yes, so far they’ve been built on that assumption — not that Akka would *guarantee* delivery in that as soon as the send() call returns you know it’s delivered, but that Akka would act the same way as a TCP socket, allowing you to send a stream of messages in order and hear when the connection breaks. Maybe that isn’t what they want to provide, but I'd find it weird, because it’s very easy to write a server with this property.
>>> 
>>> Matei
>>> 
>>> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
>>> 
>>>> Sorry if I my understanding is wrong. May be, for this particular case it might be something to do with the load/network, but, in general, are you saying that, we build these communication channels(block manager communication, task events communication, etc) assuming akka would take care of it? I somehow feel that, it's being overly optimistic. Correct me if I am wrong.
>>>> 
>>>> 
>>>> 
>>>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com> wrote:
>>>> It’s true that Akka’s delivery guarantees are in general at-most-once, but if you look at the text there it says that they differ by transport. In the previous version, I’m quite sure that except maybe in very rare circumstances or cases where we had a bug, Akka’s remote layer always kept connections up between each pair of hosts. So the guarantee was that as long as you haven’t received a “disconnected” event, your messages are being delivered, though of course when you do receive that event you don’t know which messages have really made it through unless you acked them. But that didn’t matter for our use case — from our point of view an executor was either up or down.
>>>> 
>>>> For this reason I still think it should be possible to configure Akka to do the same on 2.2. Most likely some timeouts just got lower. With large heaps you can easily get a GC pause of 60 seconds, so these timeouts should be in the minutes.
>>>> 
>>>> If for some reason this isn’t the case, then we have a bigger problem — there are *lots* of messages beyond task-finished that need to be sent reliably, including things like block manager events (a block was added / removed on this node) and commands to tell the block manager to drop data. It would be silly to implement acks at the application level for all these. But I doubt this is the case. Prashant’s observation that the standalone cluster manager stayed up is a further sign that this might be due to GC.
>>>> 
>>>> Matei
>>>> 
>>>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
>>>> 
>>>>> Hi Imran,
>>>>> Just to add, we've noticed dis-associations in a couple projects that we built(using akka 2.2.x not spark). We went to some details to find out what was happening. As Matei, suggested, Akka keeps the TCP connection open and uses that to talk to peers. We noticed that in our case, initially, we were seeing dis-associations generally at the end of keep-alive duration. So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe gets sent to inform the peer on the other side that the connection is still alive/valid. For some reason, the probe dint renew the keep-alive connection and we saw a lot of dis-associations during that time. Later, we realized this was not a pattern either. This thread contains the full history of our discussions with the Akka team. It's still open and unclear as to what was causing it for our case. 
>>>>> We tried tweaking various settings of akka(wrt heartbeats, failure detector, even plugged-in our own failure detector with no effect).
>>>>> 
>>>>> Imran - Just to clarify your point on message delivery - akka's message delivery policy is at-most-once. However, there's no guarantee for a message to be delivered to a peer. The documentation clearly explains that. http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's the responsibility of the application developer to handle cases where message is suspected to be not have been delivered. 
>>>>> 
>>>>> I hope this helps.
>>>>> 
>>>>> 
>>>>> 
>>>>> 
>>>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:
>>>>> 
>>>>> unfortunately that change wasn't the silver bullet I was hoping for.  Even with
>>>>> 1) ignoring DisassociatedEvent
>>>>> 2) executor uses ReliableProxy to send messages back to driver
>>>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>>> 
>>>>> 
>>>>> there is a lot of weird behavior.  First, there are a few DisassociatedEvents, but some that are followed by AssociatedEvents, so that seems ok.  But sometimes the re-associations are immediately followed by this:
>>>>> 
>>>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441] -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address: akka.tcp://spark@<driver>:41321] [
>>>>> akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@<driver>:41321
>>>>> Caused by: akka.remote.transport.Transport$InvalidAssociationException: The remote system has quarantined this system. No further associations to the remote system are possible until this system is restarted.
>>>>> ]
>>>>> 
>>>>> On the driver, there are messages like:
>>>>> 
>>>>> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3] [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all messages to this address will be delivered to dead letters.
>>>>> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher] Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>>> 
>>>>> 
>>>>> and when the driver does decide that the executor has been terminated, it removes the executor, but doesn't start another one.
>>>>> 
>>>>> there are a ton of messages also about messages to the block manager master ... I'm wondering if there are other parts of the system that need to use a reliable proxy (or some sort of acknowledgement).
>>>>> 
>>>>> I really don't think this was working properly even w/ previous versions of spark / akka.  I'm still learning about akka, but I think you always need an ack to be confident w/ remote communicate.  Perhaps the old version of akka just had more robust defaults or something, but I bet it could still have the same problems.  Even before, I have seen the driver thinking there were running tasks, but nothing happening on any executor -- it was just rare enough (and hard to reproduce) that I never bothered looking into it more.
>>>>> 
>>>>> I will keep digging ...
>>>>> 
>>>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com> wrote:
>>>>> BTW the problem might be the Akka failure detector settings that seem new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>>> 
>>>>> Their timeouts seem pretty aggressive by default — around 10 seconds. This can easily be too little if you have large garbage collections. We should make sure they are higher than our own node failure detection timeouts.
>>>>> 
>>>>> Matei
>>>>> 
>>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>>>>> 
>>>>>> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
>>>>>> 
>>>>>> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
>>>>>> 
>>>>>> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
>>>>>> 
>>>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>>> 
>>>>>> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
>>>>>> 
>>>>>> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
>>>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>>> 
>>>>>> imran
>>>>> 
>>>>> 
>>>>> 
>>>>> -- 
>>>>> It's just about how deep your longing is!
>>>> 
>>>> 
>>>> 
>>>> 
>>>> -- 
>>>> It's just about how deep your longing is!
>>> 
>> 
>> 
>> 
>> 
>> -- 
>> s
> 
> 
> 
> 
> -- 
> s


Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
We may no longer need to track disassociation and IMHO use the *improved*
feature in akka 2.2.x called remote death watch. Which lets us acknowledge
a remote death both in case of a natural demise and accidental deaths. This
was not the case with remote death watch in previous akka releases. Please
take a closer look at the patch at
https://github.com/apache/incubator-spark/pull/163 and let us know.

This patch does not make disassociation disappear, they are added to akka
as such but gives us sufficient knobs to tune things as to when they occur.
Don't forget to tune those extra properties apart from other timeouts.





On Sat, Nov 2, 2013 at 11:08 AM, Matei Zaharia <ma...@gmail.com>wrote:

> Prashant, the problem seems to be that messages sent while we’re
> disassociated are lost. I think we’d have to just prevent disassociation
> altogether, or replace all remote actor refs with the reliable proxies
> (which sounds painful).
>
> Matei
>
> On Nov 1, 2013, at 7:53 PM, Prashant Sharma <sc...@gmail.com> wrote:
>
> Hey Matei and Imran,
>
> I think may be we can solve the problem without downgrading to 2.1.0 may
> by capturing dissociation and then setting a timeout if it associates again
> we keep moving else we shutdown the executor. This timeout can ofcourse be
> configurable.
>
> Thoughts ?
>
>
> On Sat, Nov 2, 2013 at 3:29 AM, Matei Zaharia <ma...@gmail.com>wrote:
>
>> Hey Imran,
>>
>> Good to know that Akka 2.1 handles this — that at least will give us a
>> start.
>>
>> In the old code, executors certainly did get flagged as “down”
>> occasionally, but that was due to a timeout we controlled (we keep sending
>> heartbeats back and forth to track them). The timeout used to be smaller
>> and usually the reason to exceed it was GC. However, if Akka 2.2 can
>> sometimes drop the connections itself, this is a problem and we either have
>> to use the reliable proxies for everything or see if we can configure it
>> otherwise. Anyway, we’ll definitely look into it.
>>
>> Matei
>>
>> On Nov 1, 2013, at 1:09 PM, Imran Rashid <im...@quantifind.com> wrote:
>>
>> I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm
>> going to run my tests a few more times , but I'd really have expected to
>> see a failure by now w/ the 2.2.3 version.
>>
>> I'll submit a patch shortly (need to fix some compile errors in streaming
>> still).
>>
>> Matei -- I think I realize now that when you were talking about the
>> expectation of a tcp connection staying alive, you were explaining why this
>> is *not* a bug in the current release.  You wouldn't end up in a situation
>> where the executor thinks it finished the task, but the driver doesn't know
>> about it, b/c if the connection dies, the executor wil get restarted.  That
>> makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of
>> things change.  I was probably wrong about seeing that problem in previous
>> releases -- it was just a vague recollection, which fit my current
>> theories, so I jumped to conclusions.
>>
>> thanks everyone
>>
>>
>>
>> On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> thanks everyone for all of the input.
>>>
>>> Matei: makes a lot more sense with your explanation of spark's expected
>>> behavior of tcp, I can see why this makes sense now.  But, to show my total
>>> ignorance here, I'm wondering that when the connection does break, are you
>>> sure all of your messages that you thought you sent before the break were
>>> received?  I'm guessing that you don't.  Which is fine, if the response to
>>> that is to have the executor just die completely, and restart.  that was
>>> the behavior I was initially observing with the code on the 2.10 branch,
>>> where the executor handles a DisassociatedEvent explicitly, and dies.
>>>
>>> But -- is that the behavior we want?  do we want it to be robust to tcp
>>> connections breaking, without having to completely restart the executor?
>>> you might say that dying & restarting will lead to correct behavior, even
>>> if its inefficient.  But sometimes, I've seen restarts so frequently that
>>> no progress is made.
>>>
>>> I don't see why this changed w/ the different versions of akka -- I
>>> don't see any relevant configuration settings that would change how
>>> "strongly" tcp tries to keep the connection alive, but I may be missing
>>> something.  But it does seem like the netty configuration options have
>>> changed completely between the two versions:
>>>
>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
>>> vs
>>> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
>>>
>>> btw, akka 2.1.0 also has been built for scala 2.10:
>>>
>>> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
>>> and its netty configuration is closer to 2.0.5:
>>> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
>>>
>>> perhaps someone more knowledge then me about netty & tcp can look
>>> through the changes and decide what the right changes are.
>>>
>>> Prashant said:
>>> >Before we conclude something about reliable messaging, I want you to
>>> for once consider other possibilities like >actual network reconnection and
>>> may be a GC pause ? Try connecting something like jconsole (or alike ) and
>>> >see what happens on the driver and executor.
>>> >
>>> >My doubt are since we are using standalone mode where even master and
>>> worker are also actors then if we see >a weird behaviour on the executor
>>> and driver then Why not on master and worker too ? They should also break
>>> >away from each other. For this reason, I am doubting our conclusions and
>>> may be if we narrow down the >problem first before we conclude something.
>>> It is a regression in akka 2.2.3 it uses more memory than it used to >be in
>>> 2.1.x.
>>> >See https://github.com/akka/akka/issues/1810
>>>
>>>
>>> Well, there could easily be the same problem with dropped connections
>>> between master & worker -- they just communicate so little, it doesn't
>>> really matter.  The odds that a message gets dropped between them is very
>>> low, only because there are barely any messages.
>>>
>>> I completely agree that the problem could be because of a contention, or
>>> gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available
>>> on each box, and 90g out of 125g memory.  I've looked at gc a little with
>>> jstat, and I did see some gc pauses but nothing ridiculous.
>>>
>>> But, I think the question remains.  Suppose it is gc pauses, etc. that
>>> cause the disassociation events; what do we do to fix it?  How can we
>>> diagnose the problem, and figure out which of the configuration variables
>>> to tune?  clearly, there *will be* long gc pauses, and the networking layer
>>> needs to be able to deal with them.
>>>
>>> still I understand your desire to see if that might be the cause of the
>>> problem in this particular case, so I will dig a little more.
>>>
>>>
>>> (btw, should I move this thread to the dev list now?  it is getting into
>>> the nitty-gritty of implementation ...)
>>>
>>> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com>wrote:
>>>
>>>> Yes, so far they’ve been built on that assumption — not that Akka would
>>>> *guarantee* delivery in that as soon as the send() call returns you know
>>>> it’s delivered, but that Akka would act the same way as a TCP socket,
>>>> allowing you to send a stream of messages in order and hear when the
>>>> connection breaks. Maybe that isn’t what they want to provide, but I'd find
>>>> it weird, because it’s very easy to write a server with this property.
>>>>
>>>> Matei
>>>>
>>>> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <
>>>> sri.rams85@gmail.com> wrote:
>>>>
>>>> Sorry if I my understanding is wrong. May be, for this particular case
>>>> it might be something to do with the load/network, but, in general, are you
>>>> saying that, we build these communication channels(block manager
>>>> communication, task events communication, etc) assuming akka would take
>>>> care of it? I somehow feel that, it's being overly optimistic. Correct me
>>>> if I am wrong.
>>>>
>>>>
>>>>
>>>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <matei.zaharia@gmail.com
>>>> > wrote:
>>>>
>>>>> It’s true that Akka’s delivery guarantees are in general at-most-once,
>>>>> but if you look at the text there it says that they differ by transport. In
>>>>> the previous version, I’m quite sure that except maybe in very rare
>>>>> circumstances or cases where we had a bug, Akka’s remote layer always kept
>>>>> connections up between each pair of hosts. So the guarantee was that as
>>>>> long as you haven’t received a “disconnected” event, your messages are
>>>>> being delivered, though of course when you do receive that event you don’t
>>>>> know which messages have really made it through unless you acked them. But
>>>>> that didn’t matter for our use case — from our point of view an executor
>>>>> was either up or down.
>>>>>
>>>>> For this reason I still think it should be possible to configure Akka
>>>>> to do the same on 2.2. Most likely some timeouts just got lower. With large
>>>>> heaps you can easily get a GC pause of 60 seconds, so these timeouts should
>>>>> be in the minutes.
>>>>>
>>>>> If for some reason this isn’t the case, then we have a bigger problem
>>>>> — there are *lots* of messages beyond task-finished that need to be sent
>>>>> reliably, including things like block manager events (a block was added /
>>>>> removed on this node) and commands to tell the block manager to drop data.
>>>>> It would be silly to implement acks at the application level for all these.
>>>>> But I doubt this is the case. Prashant’s observation that the standalone
>>>>> cluster manager stayed up is a further sign that this might be due to GC.
>>>>>
>>>>> Matei
>>>>>
>>>>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <
>>>>> sri.rams85@gmail.com> wrote:
>>>>>
>>>>> Hi Imran,
>>>>> Just to add, we've noticed dis-associations in a couple projects that
>>>>> we built(using akka 2.2.x not spark). We went to some details to find out
>>>>> what was happening. As Matei, suggested, Akka keeps the TCP connection open
>>>>> and uses that to talk to peers. We noticed that in our case, initially, we
>>>>> were seeing dis-associations generally at the end of keep-alive duration.
>>>>> So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe
>>>>> gets sent to inform the peer on the other side that the connection is still
>>>>> alive/valid. For some reason, the probe dint renew the keep-alive
>>>>> connection and we saw a lot of dis-associations during that time. Later, we
>>>>> realized this was not a pattern either. This thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains the full history of our discussions with the Akka team. It's still
>>>>> open and unclear as to what was causing it for our case.
>>>>> We tried tweaking various settings of akka(wrt heartbeats, failure
>>>>> detector, even plugged-in our own failure detector with no effect).
>>>>>
>>>>> Imran - Just to clarify your point on message delivery - akka's
>>>>> message delivery policy is at-most-once. However, there's no guarantee for
>>>>> a message to be delivered to a peer. The documentation clearly explains
>>>>> that.
>>>>> http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
>>>>> the responsibility of the application developer to handle cases where
>>>>> message is suspected to be not have been delivered.
>>>>>
>>>>> I hope this helps.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>>
>>>>>> unfortunately that change wasn't the silver bullet I was hoping for.
>>>>>> Even with
>>>>>> 1) ignoring DisassociatedEvent
>>>>>> 2) executor uses ReliableProxy to send messages back to driver
>>>>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>>>>
>>>>>>
>>>>>> there is a lot of weird behavior.  First, there are a few
>>>>>> DisassociatedEvents, but some that are followed by AssociatedEvents, so
>>>>>> that seems ok.  But sometimes the re-associations are immediately followed
>>>>>> by this:
>>>>>>
>>>>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
>>>>>> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
>>>>>> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
>>>>>> akka.tcp://spark@<driver>:41321] [
>>>>>> akka.remote.InvalidAssociation: Invalid address:
>>>>>> akka.tcp://spark@<driver>:41321
>>>>>> Caused by:
>>>>>> akka.remote.transport.Transport$InvalidAssociationException: The remote
>>>>>> system has quarantined this system. No further associations to the remote
>>>>>> system are possible until this system is restarted.
>>>>>> ]
>>>>>>
>>>>>> On the driver, there are messages like:
>>>>>>
>>>>>> [INFO] [10/31/2013 18:51:07.838]
>>>>>> [spark-akka.actor.default-dispatcher-3] [Remoting] Address [
>>>>>> akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all
>>>>>> messages to this address will be delivered to dead letters.
>>>>>> [WARN] [10/31/2013 18:51:10.845]
>>>>>> [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher]
>>>>>> Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>>>>
>>>>>>
>>>>>> and when the driver does decide that the executor has been
>>>>>> terminated, it removes the executor, but doesn't start another one.
>>>>>>
>>>>>> there are a ton of messages also about messages to the block manager
>>>>>> master ... I'm wondering if there are other parts of the system that need
>>>>>> to use a reliable proxy (or some sort of acknowledgement).
>>>>>>
>>>>>> I really don't think this was working properly even w/ previous
>>>>>> versions of spark / akka.  I'm still learning about akka, but I think you
>>>>>> always need an ack to be confident w/ remote communicate.  Perhaps the old
>>>>>> version of akka just had more robust defaults or something, but I bet it
>>>>>> could still have the same problems.  Even before, I have seen the driver
>>>>>> thinking there were running tasks, but nothing happening on any executor --
>>>>>> it was just rare enough (and hard to reproduce) that I never bothered
>>>>>> looking into it more.
>>>>>>
>>>>>> I will keep digging ...
>>>>>>
>>>>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <
>>>>>> matei.zaharia@gmail.com> wrote:
>>>>>>
>>>>>>> BTW the problem might be the Akka failure detector settings that
>>>>>>> seem new in 2.2:
>>>>>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>>>>>
>>>>>>> Their timeouts seem pretty aggressive by default — around 10
>>>>>>> seconds. This can easily be too little if you have large garbage
>>>>>>> collections. We should make sure they are higher than our own node failure
>>>>>>> detection timeouts.
>>>>>>>
>>>>>>> Matei
>>>>>>>
>>>>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> pretty sure I found the problem -- two problems actually.  And I
>>>>>>> think one of them has been a general lurking problem w/ spark for a while.
>>>>>>>
>>>>>>> 1)  we should ignore disassociation events, as you suggested
>>>>>>> earlier.  They seem to just indicate a temporary problem, and can generally
>>>>>>> be ignored.  I've found that they're regularly followed by
>>>>>>> AssociatedEvents, and it seems communication really works fine at that
>>>>>>> point.
>>>>>>>
>>>>>>> 2) Task finished messages get lost.  When this message gets sent, we
>>>>>>> dont' know it actually gets there:
>>>>>>>
>>>>>>>
>>>>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>>>>
>>>>>>> (this is so incredible, I feel I must be overlooking something --
>>>>>>> but there is no ack somewhere else that I'm overlooking, is there??)  So,
>>>>>>> after the patch, spark wasn't hanging b/c of the unhandled
>>>>>>> DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished
>>>>>>> messages that never get received by the driver.  So the driver is waiting
>>>>>>> for some tasks to finish, but the executors think they are all done.
>>>>>>>
>>>>>>> I'm gonna add the reliable proxy pattern for this particular
>>>>>>> interaction and see if its fixes the problem
>>>>>>>
>>>>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>>>>
>>>>>>> imran
>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> It's just about how deep your longing is!
>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> It's just about how deep your longing is!
>>>>
>>>>
>>
>>
>
>
> --
> s
>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
Prashant, the problem seems to be that messages sent while we’re disassociated are lost. I think we’d have to just prevent disassociation altogether, or replace all remote actor refs with the reliable proxies (which sounds painful).

Matei

On Nov 1, 2013, at 7:53 PM, Prashant Sharma <sc...@gmail.com> wrote:

> Hey Matei and Imran,
> 
> I think may be we can solve the problem without downgrading to 2.1.0 may by capturing dissociation and then setting a timeout if it associates again we keep moving else we shutdown the executor. This timeout can ofcourse be configurable. 
> 
> Thoughts ?
> 
> 
> On Sat, Nov 2, 2013 at 3:29 AM, Matei Zaharia <ma...@gmail.com> wrote:
> Hey Imran,
> 
> Good to know that Akka 2.1 handles this — that at least will give us a start.
> 
> In the old code, executors certainly did get flagged as “down” occasionally, but that was due to a timeout we controlled (we keep sending heartbeats back and forth to track them). The timeout used to be smaller and usually the reason to exceed it was GC. However, if Akka 2.2 can sometimes drop the connections itself, this is a problem and we either have to use the reliable proxies for everything or see if we can configure it otherwise. Anyway, we’ll definitely look into it.
> 
> Matei
> 
> On Nov 1, 2013, at 1:09 PM, Imran Rashid <im...@quantifind.com> wrote:
> 
>> I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm going to run my tests a few more times , but I'd really have expected to see a failure by now w/ the 2.2.3 version.
>> 
>> I'll submit a patch shortly (need to fix some compile errors in streaming still).
>> 
>> Matei -- I think I realize now that when you were talking about the expectation of a tcp connection staying alive, you were explaining why this is *not* a bug in the current release.  You wouldn't end up in a situation where the executor thinks it finished the task, but the driver doesn't know about it, b/c if the connection dies, the executor wil get restarted.  That makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of things change.  I was probably wrong about seeing that problem in previous releases -- it was just a vague recollection, which fit my current theories, so I jumped to conclusions.
>> 
>> thanks everyone
>> 
>> 
>> 
>> On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com> wrote:
>> thanks everyone for all of the input.
>> 
>> Matei: makes a lot more sense with your explanation of spark's expected behavior of tcp, I can see why this makes sense now.  But, to show my total ignorance here, I'm wondering that when the connection does break, are you sure all of your messages that you thought you sent before the break were received?  I'm guessing that you don't.  Which is fine, if the response to that is to have the executor just die completely, and restart.  that was the behavior I was initially observing with the code on the 2.10 branch, where the executor handles a DisassociatedEvent explicitly, and dies.
>> 
>> But -- is that the behavior we want?  do we want it to be robust to tcp connections breaking, without having to completely restart the executor?  you might say that dying & restarting will lead to correct behavior, even if its inefficient.  But sometimes, I've seen restarts so frequently that no progress is made.
>> 
>> I don't see why this changed w/ the different versions of akka -- I don't see any relevant configuration settings that would change how "strongly" tcp tries to keep the connection alive, but I may be missing something.  But it does seem like the netty configuration options have changed completely between the two versions:
>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
>> vs
>> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
>> 
>> btw, akka 2.1.0 also has been built for scala 2.10:
>> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
>> and its netty configuration is closer to 2.0.5:
>> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
>> 
>> perhaps someone more knowledge then me about netty & tcp can look through the changes and decide what the right changes are.
>> 
>> Prashant said:
>> >Before we conclude something about reliable messaging, I want you to for once consider other possibilities like >actual network reconnection and may be a GC pause ? Try connecting something like jconsole (or alike ) and >see what happens on the driver and executor.
>> >
>> >My doubt are since we are using standalone mode where even master and worker are also actors then if we see >a weird behaviour on the executor and driver then Why not on master and worker too ? They should also break >away from each other. For this reason, I am doubting our conclusions and may be if we narrow down the >problem first before we conclude something. It is a regression in akka 2.2.3 it uses more memory than it used to >be in 2.1.x.  
>> >See https://github.com/akka/akka/issues/1810 
>> 
>> 
>> Well, there could easily be the same problem with dropped connections between master & worker -- they just communicate so little, it doesn't really matter.  The odds that a message gets dropped between them is very low, only because there are barely any messages.
>> 
>> I completely agree that the problem could be because of a contention, or gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available on each box, and 90g out of 125g memory.  I've looked at gc a little with jstat, and I did see some gc pauses but nothing ridiculous.
>> 
>> But, I think the question remains.  Suppose it is gc pauses, etc. that cause the disassociation events; what do we do to fix it?  How can we diagnose the problem, and figure out which of the configuration variables to tune?  clearly, there *will be* long gc pauses, and the networking layer needs to be able to deal with them.
>> 
>> still I understand your desire to see if that might be the cause of the problem in this particular case, so I will dig a little more.
>> 
>> 
>> (btw, should I move this thread to the dev list now?  it is getting into the nitty-gritty of implementation ...)
>> 
>> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com> wrote:
>> Yes, so far they’ve been built on that assumption — not that Akka would *guarantee* delivery in that as soon as the send() call returns you know it’s delivered, but that Akka would act the same way as a TCP socket, allowing you to send a stream of messages in order and hear when the connection breaks. Maybe that isn’t what they want to provide, but I'd find it weird, because it’s very easy to write a server with this property.
>> 
>> Matei
>> 
>> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
>> 
>>> Sorry if I my understanding is wrong. May be, for this particular case it might be something to do with the load/network, but, in general, are you saying that, we build these communication channels(block manager communication, task events communication, etc) assuming akka would take care of it? I somehow feel that, it's being overly optimistic. Correct me if I am wrong.
>>> 
>>> 
>>> 
>>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com> wrote:
>>> It’s true that Akka’s delivery guarantees are in general at-most-once, but if you look at the text there it says that they differ by transport. In the previous version, I’m quite sure that except maybe in very rare circumstances or cases where we had a bug, Akka’s remote layer always kept connections up between each pair of hosts. So the guarantee was that as long as you haven’t received a “disconnected” event, your messages are being delivered, though of course when you do receive that event you don’t know which messages have really made it through unless you acked them. But that didn’t matter for our use case — from our point of view an executor was either up or down.
>>> 
>>> For this reason I still think it should be possible to configure Akka to do the same on 2.2. Most likely some timeouts just got lower. With large heaps you can easily get a GC pause of 60 seconds, so these timeouts should be in the minutes.
>>> 
>>> If for some reason this isn’t the case, then we have a bigger problem — there are *lots* of messages beyond task-finished that need to be sent reliably, including things like block manager events (a block was added / removed on this node) and commands to tell the block manager to drop data. It would be silly to implement acks at the application level for all these. But I doubt this is the case. Prashant’s observation that the standalone cluster manager stayed up is a further sign that this might be due to GC.
>>> 
>>> Matei
>>> 
>>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
>>> 
>>>> Hi Imran,
>>>> Just to add, we've noticed dis-associations in a couple projects that we built(using akka 2.2.x not spark). We went to some details to find out what was happening. As Matei, suggested, Akka keeps the TCP connection open and uses that to talk to peers. We noticed that in our case, initially, we were seeing dis-associations generally at the end of keep-alive duration. So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe gets sent to inform the peer on the other side that the connection is still alive/valid. For some reason, the probe dint renew the keep-alive connection and we saw a lot of dis-associations during that time. Later, we realized this was not a pattern either. This thread contains the full history of our discussions with the Akka team. It's still open and unclear as to what was causing it for our case. 
>>>> We tried tweaking various settings of akka(wrt heartbeats, failure detector, even plugged-in our own failure detector with no effect).
>>>> 
>>>> Imran - Just to clarify your point on message delivery - akka's message delivery policy is at-most-once. However, there's no guarantee for a message to be delivered to a peer. The documentation clearly explains that. http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's the responsibility of the application developer to handle cases where message is suspected to be not have been delivered. 
>>>> 
>>>> I hope this helps.
>>>> 
>>>> 
>>>> 
>>>> 
>>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:
>>>> 
>>>> unfortunately that change wasn't the silver bullet I was hoping for.  Even with
>>>> 1) ignoring DisassociatedEvent
>>>> 2) executor uses ReliableProxy to send messages back to driver
>>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>> 
>>>> 
>>>> there is a lot of weird behavior.  First, there are a few DisassociatedEvents, but some that are followed by AssociatedEvents, so that seems ok.  But sometimes the re-associations are immediately followed by this:
>>>> 
>>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441] -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address: akka.tcp://spark@<driver>:41321] [
>>>> akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@<driver>:41321
>>>> Caused by: akka.remote.transport.Transport$InvalidAssociationException: The remote system has quarantined this system. No further associations to the remote system are possible until this system is restarted.
>>>> ]
>>>> 
>>>> On the driver, there are messages like:
>>>> 
>>>> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3] [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all messages to this address will be delivered to dead letters.
>>>> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher] Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>> 
>>>> 
>>>> and when the driver does decide that the executor has been terminated, it removes the executor, but doesn't start another one.
>>>> 
>>>> there are a ton of messages also about messages to the block manager master ... I'm wondering if there are other parts of the system that need to use a reliable proxy (or some sort of acknowledgement).
>>>> 
>>>> I really don't think this was working properly even w/ previous versions of spark / akka.  I'm still learning about akka, but I think you always need an ack to be confident w/ remote communicate.  Perhaps the old version of akka just had more robust defaults or something, but I bet it could still have the same problems.  Even before, I have seen the driver thinking there were running tasks, but nothing happening on any executor -- it was just rare enough (and hard to reproduce) that I never bothered looking into it more.
>>>> 
>>>> I will keep digging ...
>>>> 
>>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com> wrote:
>>>> BTW the problem might be the Akka failure detector settings that seem new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>> 
>>>> Their timeouts seem pretty aggressive by default — around 10 seconds. This can easily be too little if you have large garbage collections. We should make sure they are higher than our own node failure detection timeouts.
>>>> 
>>>> Matei
>>>> 
>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>>>> 
>>>>> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
>>>>> 
>>>>> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
>>>>> 
>>>>> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
>>>>> 
>>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>> 
>>>>> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
>>>>> 
>>>>> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
>>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>> 
>>>>> imran
>>>> 
>>>> 
>>>> 
>>>> -- 
>>>> It's just about how deep your longing is!
>>> 
>>> 
>>> 
>>> 
>>> -- 
>>> It's just about how deep your longing is!
>> 
> 
> 
> 
> 
> -- 
> s


Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Hey Matei and Imran,

I think may be we can solve the problem without downgrading to 2.1.0 may by
capturing dissociation and then setting a timeout if it associates again we
keep moving else we shutdown the executor. This timeout can ofcourse be
configurable.

Thoughts ?


On Sat, Nov 2, 2013 at 3:29 AM, Matei Zaharia <ma...@gmail.com>wrote:

> Hey Imran,
>
> Good to know that Akka 2.1 handles this — that at least will give us a
> start.
>
> In the old code, executors certainly did get flagged as “down”
> occasionally, but that was due to a timeout we controlled (we keep sending
> heartbeats back and forth to track them). The timeout used to be smaller
> and usually the reason to exceed it was GC. However, if Akka 2.2 can
> sometimes drop the connections itself, this is a problem and we either have
> to use the reliable proxies for everything or see if we can configure it
> otherwise. Anyway, we’ll definitely look into it.
>
> Matei
>
> On Nov 1, 2013, at 1:09 PM, Imran Rashid <im...@quantifind.com> wrote:
>
> I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm
> going to run my tests a few more times , but I'd really have expected to
> see a failure by now w/ the 2.2.3 version.
>
> I'll submit a patch shortly (need to fix some compile errors in streaming
> still).
>
> Matei -- I think I realize now that when you were talking about the
> expectation of a tcp connection staying alive, you were explaining why this
> is *not* a bug in the current release.  You wouldn't end up in a situation
> where the executor thinks it finished the task, but the driver doesn't know
> about it, b/c if the connection dies, the executor wil get restarted.  That
> makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of
> things change.  I was probably wrong about seeing that problem in previous
> releases -- it was just a vague recollection, which fit my current
> theories, so I jumped to conclusions.
>
> thanks everyone
>
>
>
> On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com> wrote:
>
>> thanks everyone for all of the input.
>>
>> Matei: makes a lot more sense with your explanation of spark's expected
>> behavior of tcp, I can see why this makes sense now.  But, to show my total
>> ignorance here, I'm wondering that when the connection does break, are you
>> sure all of your messages that you thought you sent before the break were
>> received?  I'm guessing that you don't.  Which is fine, if the response to
>> that is to have the executor just die completely, and restart.  that was
>> the behavior I was initially observing with the code on the 2.10 branch,
>> where the executor handles a DisassociatedEvent explicitly, and dies.
>>
>> But -- is that the behavior we want?  do we want it to be robust to tcp
>> connections breaking, without having to completely restart the executor?
>> you might say that dying & restarting will lead to correct behavior, even
>> if its inefficient.  But sometimes, I've seen restarts so frequently that
>> no progress is made.
>>
>> I don't see why this changed w/ the different versions of akka -- I don't
>> see any relevant configuration settings that would change how "strongly"
>> tcp tries to keep the connection alive, but I may be missing something.
>> But it does seem like the netty configuration options have changed
>> completely between the two versions:
>>
>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
>> vs
>> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
>>
>> btw, akka 2.1.0 also has been built for scala 2.10:
>>
>> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
>> and its netty configuration is closer to 2.0.5:
>> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
>>
>> perhaps someone more knowledge then me about netty & tcp can look through
>> the changes and decide what the right changes are.
>>
>> Prashant said:
>> >Before we conclude something about reliable messaging, I want you to for
>> once consider other possibilities like >actual network reconnection and may
>> be a GC pause ? Try connecting something like jconsole (or alike ) and >see
>> what happens on the driver and executor.
>> >
>> >My doubt are since we are using standalone mode where even master and
>> worker are also actors then if we see >a weird behaviour on the executor
>> and driver then Why not on master and worker too ? They should also break
>> >away from each other. For this reason, I am doubting our conclusions and
>> may be if we narrow down the >problem first before we conclude something.
>> It is a regression in akka 2.2.3 it uses more memory than it used to >be in
>> 2.1.x.
>> >See https://github.com/akka/akka/issues/1810
>>
>>
>> Well, there could easily be the same problem with dropped connections
>> between master & worker -- they just communicate so little, it doesn't
>> really matter.  The odds that a message gets dropped between them is very
>> low, only because there are barely any messages.
>>
>> I completely agree that the problem could be because of a contention, or
>> gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available
>> on each box, and 90g out of 125g memory.  I've looked at gc a little with
>> jstat, and I did see some gc pauses but nothing ridiculous.
>>
>> But, I think the question remains.  Suppose it is gc pauses, etc. that
>> cause the disassociation events; what do we do to fix it?  How can we
>> diagnose the problem, and figure out which of the configuration variables
>> to tune?  clearly, there *will be* long gc pauses, and the networking layer
>> needs to be able to deal with them.
>>
>> still I understand your desire to see if that might be the cause of the
>> problem in this particular case, so I will dig a little more.
>>
>>
>> (btw, should I move this thread to the dev list now?  it is getting into
>> the nitty-gritty of implementation ...)
>>
>> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com>wrote:
>>
>>> Yes, so far they’ve been built on that assumption — not that Akka would
>>> *guarantee* delivery in that as soon as the send() call returns you know
>>> it’s delivered, but that Akka would act the same way as a TCP socket,
>>> allowing you to send a stream of messages in order and hear when the
>>> connection breaks. Maybe that isn’t what they want to provide, but I'd find
>>> it weird, because it’s very easy to write a server with this property.
>>>
>>> Matei
>>>
>>> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <
>>> sri.rams85@gmail.com> wrote:
>>>
>>> Sorry if I my understanding is wrong. May be, for this particular case
>>> it might be something to do with the load/network, but, in general, are you
>>> saying that, we build these communication channels(block manager
>>> communication, task events communication, etc) assuming akka would take
>>> care of it? I somehow feel that, it's being overly optimistic. Correct me
>>> if I am wrong.
>>>
>>>
>>>
>>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com>wrote:
>>>
>>>> It’s true that Akka’s delivery guarantees are in general at-most-once,
>>>> but if you look at the text there it says that they differ by transport. In
>>>> the previous version, I’m quite sure that except maybe in very rare
>>>> circumstances or cases where we had a bug, Akka’s remote layer always kept
>>>> connections up between each pair of hosts. So the guarantee was that as
>>>> long as you haven’t received a “disconnected” event, your messages are
>>>> being delivered, though of course when you do receive that event you don’t
>>>> know which messages have really made it through unless you acked them. But
>>>> that didn’t matter for our use case — from our point of view an executor
>>>> was either up or down.
>>>>
>>>> For this reason I still think it should be possible to configure Akka
>>>> to do the same on 2.2. Most likely some timeouts just got lower. With large
>>>> heaps you can easily get a GC pause of 60 seconds, so these timeouts should
>>>> be in the minutes.
>>>>
>>>> If for some reason this isn’t the case, then we have a bigger problem —
>>>> there are *lots* of messages beyond task-finished that need to be sent
>>>> reliably, including things like block manager events (a block was added /
>>>> removed on this node) and commands to tell the block manager to drop data.
>>>> It would be silly to implement acks at the application level for all these.
>>>> But I doubt this is the case. Prashant’s observation that the standalone
>>>> cluster manager stayed up is a further sign that this might be due to GC.
>>>>
>>>> Matei
>>>>
>>>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <
>>>> sri.rams85@gmail.com> wrote:
>>>>
>>>> Hi Imran,
>>>> Just to add, we've noticed dis-associations in a couple projects that
>>>> we built(using akka 2.2.x not spark). We went to some details to find out
>>>> what was happening. As Matei, suggested, Akka keeps the TCP connection open
>>>> and uses that to talk to peers. We noticed that in our case, initially, we
>>>> were seeing dis-associations generally at the end of keep-alive duration.
>>>> So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe
>>>> gets sent to inform the peer on the other side that the connection is still
>>>> alive/valid. For some reason, the probe dint renew the keep-alive
>>>> connection and we saw a lot of dis-associations during that time. Later, we
>>>> realized this was not a pattern either. This thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains the full history of our discussions with the Akka team. It's still
>>>> open and unclear as to what was causing it for our case.
>>>> We tried tweaking various settings of akka(wrt heartbeats, failure
>>>> detector, even plugged-in our own failure detector with no effect).
>>>>
>>>> Imran - Just to clarify your point on message delivery - akka's message
>>>> delivery policy is at-most-once. However, there's no guarantee for a
>>>> message to be delivered to a peer. The documentation clearly explains that.
>>>> http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
>>>> the responsibility of the application developer to handle cases where
>>>> message is suspected to be not have been delivered.
>>>>
>>>> I hope this helps.
>>>>
>>>>
>>>>
>>>>
>>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>>
>>>>> unfortunately that change wasn't the silver bullet I was hoping for.
>>>>> Even with
>>>>> 1) ignoring DisassociatedEvent
>>>>> 2) executor uses ReliableProxy to send messages back to driver
>>>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>>>
>>>>>
>>>>> there is a lot of weird behavior.  First, there are a few
>>>>> DisassociatedEvents, but some that are followed by AssociatedEvents, so
>>>>> that seems ok.  But sometimes the re-associations are immediately followed
>>>>> by this:
>>>>>
>>>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
>>>>> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
>>>>> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
>>>>> akka.tcp://spark@<driver>:41321] [
>>>>> akka.remote.InvalidAssociation: Invalid address:
>>>>> akka.tcp://spark@<driver>:41321
>>>>> Caused by:
>>>>> akka.remote.transport.Transport$InvalidAssociationException: The remote
>>>>> system has quarantined this system. No further associations to the remote
>>>>> system are possible until this system is restarted.
>>>>> ]
>>>>>
>>>>> On the driver, there are messages like:
>>>>>
>>>>> [INFO] [10/31/2013 18:51:07.838]
>>>>> [spark-akka.actor.default-dispatcher-3] [Remoting] Address [
>>>>> akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all
>>>>> messages to this address will be delivered to dead letters.
>>>>> [WARN] [10/31/2013 18:51:10.845]
>>>>> [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher]
>>>>> Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>>>
>>>>>
>>>>> and when the driver does decide that the executor has been terminated,
>>>>> it removes the executor, but doesn't start another one.
>>>>>
>>>>> there are a ton of messages also about messages to the block manager
>>>>> master ... I'm wondering if there are other parts of the system that need
>>>>> to use a reliable proxy (or some sort of acknowledgement).
>>>>>
>>>>> I really don't think this was working properly even w/ previous
>>>>> versions of spark / akka.  I'm still learning about akka, but I think you
>>>>> always need an ack to be confident w/ remote communicate.  Perhaps the old
>>>>> version of akka just had more robust defaults or something, but I bet it
>>>>> could still have the same problems.  Even before, I have seen the driver
>>>>> thinking there were running tasks, but nothing happening on any executor --
>>>>> it was just rare enough (and hard to reproduce) that I never bothered
>>>>> looking into it more.
>>>>>
>>>>> I will keep digging ...
>>>>>
>>>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <
>>>>> matei.zaharia@gmail.com> wrote:
>>>>>
>>>>>> BTW the problem might be the Akka failure detector settings that seem
>>>>>> new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>>>>
>>>>>> Their timeouts seem pretty aggressive by default — around 10 seconds.
>>>>>> This can easily be too little if you have large garbage collections. We
>>>>>> should make sure they are higher than our own node failure detection
>>>>>> timeouts.
>>>>>>
>>>>>> Matei
>>>>>>
>>>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com>
>>>>>> wrote:
>>>>>>
>>>>>> pretty sure I found the problem -- two problems actually.  And I
>>>>>> think one of them has been a general lurking problem w/ spark for a while.
>>>>>>
>>>>>> 1)  we should ignore disassociation events, as you suggested
>>>>>> earlier.  They seem to just indicate a temporary problem, and can generally
>>>>>> be ignored.  I've found that they're regularly followed by
>>>>>> AssociatedEvents, and it seems communication really works fine at that
>>>>>> point.
>>>>>>
>>>>>> 2) Task finished messages get lost.  When this message gets sent, we
>>>>>> dont' know it actually gets there:
>>>>>>
>>>>>>
>>>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>>>
>>>>>> (this is so incredible, I feel I must be overlooking something -- but
>>>>>> there is no ack somewhere else that I'm overlooking, is there??)  So, after
>>>>>> the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.
>>>>>> It hangs b/c the executor has sent some taskFinished messages that never
>>>>>> get received by the driver.  So the driver is waiting for some tasks to
>>>>>> finish, but the executors think they are all done.
>>>>>>
>>>>>> I'm gonna add the reliable proxy pattern for this particular
>>>>>> interaction and see if its fixes the problem
>>>>>>
>>>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>>>
>>>>>> imran
>>>>>>
>>>>>>
>>>>
>>>>
>>>> --
>>>> It's just about how deep your longing is!
>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> It's just about how deep your longing is!
>>>
>>>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
Hey Imran,

Good to know that Akka 2.1 handles this — that at least will give us a start.

In the old code, executors certainly did get flagged as “down” occasionally, but that was due to a timeout we controlled (we keep sending heartbeats back and forth to track them). The timeout used to be smaller and usually the reason to exceed it was GC. However, if Akka 2.2 can sometimes drop the connections itself, this is a problem and we either have to use the reliable proxies for everything or see if we can configure it otherwise. Anyway, we’ll definitely look into it.

Matei

On Nov 1, 2013, at 1:09 PM, Imran Rashid <im...@quantifind.com> wrote:

> I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm going to run my tests a few more times , but I'd really have expected to see a failure by now w/ the 2.2.3 version.
> 
> I'll submit a patch shortly (need to fix some compile errors in streaming still).
> 
> Matei -- I think I realize now that when you were talking about the expectation of a tcp connection staying alive, you were explaining why this is *not* a bug in the current release.  You wouldn't end up in a situation where the executor thinks it finished the task, but the driver doesn't know about it, b/c if the connection dies, the executor wil get restarted.  That makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of things change.  I was probably wrong about seeing that problem in previous releases -- it was just a vague recollection, which fit my current theories, so I jumped to conclusions.
> 
> thanks everyone
> 
> 
> 
> On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com> wrote:
> thanks everyone for all of the input.
> 
> Matei: makes a lot more sense with your explanation of spark's expected behavior of tcp, I can see why this makes sense now.  But, to show my total ignorance here, I'm wondering that when the connection does break, are you sure all of your messages that you thought you sent before the break were received?  I'm guessing that you don't.  Which is fine, if the response to that is to have the executor just die completely, and restart.  that was the behavior I was initially observing with the code on the 2.10 branch, where the executor handles a DisassociatedEvent explicitly, and dies.
> 
> But -- is that the behavior we want?  do we want it to be robust to tcp connections breaking, without having to completely restart the executor?  you might say that dying & restarting will lead to correct behavior, even if its inefficient.  But sometimes, I've seen restarts so frequently that no progress is made.
> 
> I don't see why this changed w/ the different versions of akka -- I don't see any relevant configuration settings that would change how "strongly" tcp tries to keep the connection alive, but I may be missing something.  But it does seem like the netty configuration options have changed completely between the two versions:
> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
> vs
> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
> 
> btw, akka 2.1.0 also has been built for scala 2.10:
> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
> and its netty configuration is closer to 2.0.5:
> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
> 
> perhaps someone more knowledge then me about netty & tcp can look through the changes and decide what the right changes are.
> 
> Prashant said:
> >Before we conclude something about reliable messaging, I want you to for once consider other possibilities like >actual network reconnection and may be a GC pause ? Try connecting something like jconsole (or alike ) and >see what happens on the driver and executor.
> >
> >My doubt are since we are using standalone mode where even master and worker are also actors then if we see >a weird behaviour on the executor and driver then Why not on master and worker too ? They should also break >away from each other. For this reason, I am doubting our conclusions and may be if we narrow down the >problem first before we conclude something. It is a regression in akka 2.2.3 it uses more memory than it used to >be in 2.1.x.  
> >See https://github.com/akka/akka/issues/1810 
> 
> 
> Well, there could easily be the same problem with dropped connections between master & worker -- they just communicate so little, it doesn't really matter.  The odds that a message gets dropped between them is very low, only because there are barely any messages.
> 
> I completely agree that the problem could be because of a contention, or gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available on each box, and 90g out of 125g memory.  I've looked at gc a little with jstat, and I did see some gc pauses but nothing ridiculous.
> 
> But, I think the question remains.  Suppose it is gc pauses, etc. that cause the disassociation events; what do we do to fix it?  How can we diagnose the problem, and figure out which of the configuration variables to tune?  clearly, there *will be* long gc pauses, and the networking layer needs to be able to deal with them.
> 
> still I understand your desire to see if that might be the cause of the problem in this particular case, so I will dig a little more.
> 
> 
> (btw, should I move this thread to the dev list now?  it is getting into the nitty-gritty of implementation ...)
> 
> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com> wrote:
> Yes, so far they’ve been built on that assumption — not that Akka would *guarantee* delivery in that as soon as the send() call returns you know it’s delivered, but that Akka would act the same way as a TCP socket, allowing you to send a stream of messages in order and hear when the connection breaks. Maybe that isn’t what they want to provide, but I'd find it weird, because it’s very easy to write a server with this property.
> 
> Matei
> 
> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
> 
>> Sorry if I my understanding is wrong. May be, for this particular case it might be something to do with the load/network, but, in general, are you saying that, we build these communication channels(block manager communication, task events communication, etc) assuming akka would take care of it? I somehow feel that, it's being overly optimistic. Correct me if I am wrong.
>> 
>> 
>> 
>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com> wrote:
>> It’s true that Akka’s delivery guarantees are in general at-most-once, but if you look at the text there it says that they differ by transport. In the previous version, I’m quite sure that except maybe in very rare circumstances or cases where we had a bug, Akka’s remote layer always kept connections up between each pair of hosts. So the guarantee was that as long as you haven’t received a “disconnected” event, your messages are being delivered, though of course when you do receive that event you don’t know which messages have really made it through unless you acked them. But that didn’t matter for our use case — from our point of view an executor was either up or down.
>> 
>> For this reason I still think it should be possible to configure Akka to do the same on 2.2. Most likely some timeouts just got lower. With large heaps you can easily get a GC pause of 60 seconds, so these timeouts should be in the minutes.
>> 
>> If for some reason this isn’t the case, then we have a bigger problem — there are *lots* of messages beyond task-finished that need to be sent reliably, including things like block manager events (a block was added / removed on this node) and commands to tell the block manager to drop data. It would be silly to implement acks at the application level for all these. But I doubt this is the case. Prashant’s observation that the standalone cluster manager stayed up is a further sign that this might be due to GC.
>> 
>> Matei
>> 
>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
>> 
>>> Hi Imran,
>>> Just to add, we've noticed dis-associations in a couple projects that we built(using akka 2.2.x not spark). We went to some details to find out what was happening. As Matei, suggested, Akka keeps the TCP connection open and uses that to talk to peers. We noticed that in our case, initially, we were seeing dis-associations generally at the end of keep-alive duration. So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe gets sent to inform the peer on the other side that the connection is still alive/valid. For some reason, the probe dint renew the keep-alive connection and we saw a lot of dis-associations during that time. Later, we realized this was not a pattern either. This thread contains the full history of our discussions with the Akka team. It's still open and unclear as to what was causing it for our case. 
>>> We tried tweaking various settings of akka(wrt heartbeats, failure detector, even plugged-in our own failure detector with no effect).
>>> 
>>> Imran - Just to clarify your point on message delivery - akka's message delivery policy is at-most-once. However, there's no guarantee for a message to be delivered to a peer. The documentation clearly explains that. http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's the responsibility of the application developer to handle cases where message is suspected to be not have been delivered. 
>>> 
>>> I hope this helps.
>>> 
>>> 
>>> 
>>> 
>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:
>>> 
>>> unfortunately that change wasn't the silver bullet I was hoping for.  Even with
>>> 1) ignoring DisassociatedEvent
>>> 2) executor uses ReliableProxy to send messages back to driver
>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>> 
>>> 
>>> there is a lot of weird behavior.  First, there are a few DisassociatedEvents, but some that are followed by AssociatedEvents, so that seems ok.  But sometimes the re-associations are immediately followed by this:
>>> 
>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441] -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address: akka.tcp://spark@<driver>:41321] [
>>> akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@<driver>:41321
>>> Caused by: akka.remote.transport.Transport$InvalidAssociationException: The remote system has quarantined this system. No further associations to the remote system are possible until this system is restarted.
>>> ]
>>> 
>>> On the driver, there are messages like:
>>> 
>>> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3] [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all messages to this address will be delivered to dead letters.
>>> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher] Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>> 
>>> 
>>> and when the driver does decide that the executor has been terminated, it removes the executor, but doesn't start another one.
>>> 
>>> there are a ton of messages also about messages to the block manager master ... I'm wondering if there are other parts of the system that need to use a reliable proxy (or some sort of acknowledgement).
>>> 
>>> I really don't think this was working properly even w/ previous versions of spark / akka.  I'm still learning about akka, but I think you always need an ack to be confident w/ remote communicate.  Perhaps the old version of akka just had more robust defaults or something, but I bet it could still have the same problems.  Even before, I have seen the driver thinking there were running tasks, but nothing happening on any executor -- it was just rare enough (and hard to reproduce) that I never bothered looking into it more.
>>> 
>>> I will keep digging ...
>>> 
>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com> wrote:
>>> BTW the problem might be the Akka failure detector settings that seem new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>> 
>>> Their timeouts seem pretty aggressive by default — around 10 seconds. This can easily be too little if you have large garbage collections. We should make sure they are higher than our own node failure detection timeouts.
>>> 
>>> Matei
>>> 
>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>>> 
>>>> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
>>>> 
>>>> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
>>>> 
>>>> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
>>>> 
>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>> 
>>>> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
>>>> 
>>>> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>> 
>>>> imran
>>> 
>>> 
>>> 
>>> -- 
>>> It's just about how deep your longing is!
>> 
>> 
>> 
>> 
>> -- 
>> It's just about how deep your longing is!
> 


Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
I downgraded spark to akka 2.1.0, and everything seems to work now.  I'm
going to run my tests a few more times , but I'd really have expected to
see a failure by now w/ the 2.2.3 version.

I'll submit a patch shortly (need to fix some compile errors in streaming
still).

Matei -- I think I realize now that when you were talking about the
expectation of a tcp connection staying alive, you were explaining why this
is *not* a bug in the current release.  You wouldn't end up in a situation
where the executor thinks it finished the task, but the driver doesn't know
about it, b/c if the connection dies, the executor wil get restarted.  That
makes sense.  But, it seems like if we upgrade to akka 2.2.x, a lot of
things change.  I was probably wrong about seeing that problem in previous
releases -- it was just a vague recollection, which fit my current
theories, so I jumped to conclusions.

thanks everyone



On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid <im...@quantifind.com> wrote:

> thanks everyone for all of the input.
>
> Matei: makes a lot more sense with your explanation of spark's expected
> behavior of tcp, I can see why this makes sense now.  But, to show my total
> ignorance here, I'm wondering that when the connection does break, are you
> sure all of your messages that you thought you sent before the break were
> received?  I'm guessing that you don't.  Which is fine, if the response to
> that is to have the executor just die completely, and restart.  that was
> the behavior I was initially observing with the code on the 2.10 branch,
> where the executor handles a DisassociatedEvent explicitly, and dies.
>
> But -- is that the behavior we want?  do we want it to be robust to tcp
> connections breaking, without having to completely restart the executor?
> you might say that dying & restarting will lead to correct behavior, even
> if its inefficient.  But sometimes, I've seen restarts so frequently that
> no progress is made.
>
> I don't see why this changed w/ the different versions of akka -- I don't
> see any relevant configuration settings that would change how "strongly"
> tcp tries to keep the connection alive, but I may be missing something.
> But it does seem like the netty configuration options have changed
> completely between the two versions:
> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
> vs
> http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html
>
> btw, akka 2.1.0 also has been built for scala 2.10:
>
> http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
> and its netty configuration is closer to 2.0.5:
> http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html
>
> perhaps someone more knowledge then me about netty & tcp can look through
> the changes and decide what the right changes are.
>
> Prashant said:
> >Before we conclude something about reliable messaging, I want you to for
> once consider other possibilities like >actual network reconnection and may
> be a GC pause ? Try connecting something like jconsole (or alike ) and >see
> what happens on the driver and executor.
> >
> >My doubt are since we are using standalone mode where even master and
> worker are also actors then if we see >a weird behaviour on the executor
> and driver then Why not on master and worker too ? They should also break
> >away from each other. For this reason, I am doubting our conclusions and
> may be if we narrow down the >problem first before we conclude something.
> It is a regression in akka 2.2.3 it uses more memory than it used to >be in
> 2.1.x.
> >See https://github.com/akka/akka/issues/1810
>
>
> Well, there could easily be the same problem with dropped connections
> between master & worker -- they just communicate so little, it doesn't
> really matter.  The odds that a message gets dropped between them is very
> low, only because there are barely any messages.
>
> I completely agree that the problem could be because of a contention, or
> gc pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available
> on each box, and 90g out of 125g memory.  I've looked at gc a little with
> jstat, and I did see some gc pauses but nothing ridiculous.
>
> But, I think the question remains.  Suppose it is gc pauses, etc. that
> cause the disassociation events; what do we do to fix it?  How can we
> diagnose the problem, and figure out which of the configuration variables
> to tune?  clearly, there *will be* long gc pauses, and the networking layer
> needs to be able to deal with them.
>
> still I understand your desire to see if that might be the cause of the
> problem in this particular case, so I will dig a little more.
>
>
> (btw, should I move this thread to the dev list now?  it is getting into
> the nitty-gritty of implementation ...)
>
> On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com>wrote:
>
>> Yes, so far they’ve been built on that assumption — not that Akka would
>> *guarantee* delivery in that as soon as the send() call returns you know
>> it’s delivered, but that Akka would act the same way as a TCP socket,
>> allowing you to send a stream of messages in order and hear when the
>> connection breaks. Maybe that isn’t what they want to provide, but I'd find
>> it weird, because it’s very easy to write a server with this property.
>>
>> Matei
>>
>> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <
>> sri.rams85@gmail.com> wrote:
>>
>> Sorry if I my understanding is wrong. May be, for this particular case it
>> might be something to do with the load/network, but, in general, are you
>> saying that, we build these communication channels(block manager
>> communication, task events communication, etc) assuming akka would take
>> care of it? I somehow feel that, it's being overly optimistic. Correct me
>> if I am wrong.
>>
>>
>>
>> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com>wrote:
>>
>>> It’s true that Akka’s delivery guarantees are in general at-most-once,
>>> but if you look at the text there it says that they differ by transport. In
>>> the previous version, I’m quite sure that except maybe in very rare
>>> circumstances or cases where we had a bug, Akka’s remote layer always kept
>>> connections up between each pair of hosts. So the guarantee was that as
>>> long as you haven’t received a “disconnected” event, your messages are
>>> being delivered, though of course when you do receive that event you don’t
>>> know which messages have really made it through unless you acked them. But
>>> that didn’t matter for our use case — from our point of view an executor
>>> was either up or down.
>>>
>>> For this reason I still think it should be possible to configure Akka to
>>> do the same on 2.2. Most likely some timeouts just got lower. With large
>>> heaps you can easily get a GC pause of 60 seconds, so these timeouts should
>>> be in the minutes.
>>>
>>> If for some reason this isn’t the case, then we have a bigger problem —
>>> there are *lots* of messages beyond task-finished that need to be sent
>>> reliably, including things like block manager events (a block was added /
>>> removed on this node) and commands to tell the block manager to drop data.
>>> It would be silly to implement acks at the application level for all these.
>>> But I doubt this is the case. Prashant’s observation that the standalone
>>> cluster manager stayed up is a further sign that this might be due to GC.
>>>
>>> Matei
>>>
>>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <
>>> sri.rams85@gmail.com> wrote:
>>>
>>> Hi Imran,
>>> Just to add, we've noticed dis-associations in a couple projects that we
>>> built(using akka 2.2.x not spark). We went to some details to find out what
>>> was happening. As Matei, suggested, Akka keeps the TCP connection open and
>>> uses that to talk to peers. We noticed that in our case, initially, we were
>>> seeing dis-associations generally at the end of keep-alive duration. So,
>>> when the keep-alive duration ends, at the TCP layer, a keep-alive probe
>>> gets sent to inform the peer on the other side that the connection is still
>>> alive/valid. For some reason, the probe dint renew the keep-alive
>>> connection and we saw a lot of dis-associations during that time. Later, we
>>> realized this was not a pattern either. This thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains the full history of our discussions with the Akka team. It's still
>>> open and unclear as to what was causing it for our case.
>>> We tried tweaking various settings of akka(wrt heartbeats, failure
>>> detector, even plugged-in our own failure detector with no effect).
>>>
>>> Imran - Just to clarify your point on message delivery - akka's message
>>> delivery policy is at-most-once. However, there's no guarantee for a
>>> message to be delivered to a peer. The documentation clearly explains that.
>>> http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
>>> the responsibility of the application developer to handle cases where
>>> message is suspected to be not have been delivered.
>>>
>>> I hope this helps.
>>>
>>>
>>>
>>>
>>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>>
>>>> unfortunately that change wasn't the silver bullet I was hoping for.
>>>> Even with
>>>> 1) ignoring DisassociatedEvent
>>>> 2) executor uses ReliableProxy to send messages back to driver
>>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>>
>>>>
>>>> there is a lot of weird behavior.  First, there are a few
>>>> DisassociatedEvents, but some that are followed by AssociatedEvents, so
>>>> that seems ok.  But sometimes the re-associations are immediately followed
>>>> by this:
>>>>
>>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
>>>> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
>>>> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
>>>> akka.tcp://spark@<driver>:41321] [
>>>> akka.remote.InvalidAssociation: Invalid address:
>>>> akka.tcp://spark@<driver>:41321
>>>> Caused by: akka.remote.transport.Transport$InvalidAssociationException:
>>>> The remote system has quarantined this system. No further associations to
>>>> the remote system are possible until this system is restarted.
>>>> ]
>>>>
>>>> On the driver, there are messages like:
>>>>
>>>> [INFO] [10/31/2013 18:51:07.838]
>>>> [spark-akka.actor.default-dispatcher-3] [Remoting] Address [
>>>> akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all
>>>> messages to this address will be delivered to dead letters.
>>>> [WARN] [10/31/2013 18:51:10.845]
>>>> [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher]
>>>> Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>>
>>>>
>>>> and when the driver does decide that the executor has been terminated,
>>>> it removes the executor, but doesn't start another one.
>>>>
>>>> there are a ton of messages also about messages to the block manager
>>>> master ... I'm wondering if there are other parts of the system that need
>>>> to use a reliable proxy (or some sort of acknowledgement).
>>>>
>>>> I really don't think this was working properly even w/ previous
>>>> versions of spark / akka.  I'm still learning about akka, but I think you
>>>> always need an ack to be confident w/ remote communicate.  Perhaps the old
>>>> version of akka just had more robust defaults or something, but I bet it
>>>> could still have the same problems.  Even before, I have seen the driver
>>>> thinking there were running tasks, but nothing happening on any executor --
>>>> it was just rare enough (and hard to reproduce) that I never bothered
>>>> looking into it more.
>>>>
>>>> I will keep digging ...
>>>>
>>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <matei.zaharia@gmail.com
>>>> > wrote:
>>>>
>>>>> BTW the problem might be the Akka failure detector settings that seem
>>>>> new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>>>
>>>>> Their timeouts seem pretty aggressive by default — around 10 seconds.
>>>>> This can easily be too little if you have large garbage collections. We
>>>>> should make sure they are higher than our own node failure detection
>>>>> timeouts.
>>>>>
>>>>> Matei
>>>>>
>>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com>
>>>>> wrote:
>>>>>
>>>>> pretty sure I found the problem -- two problems actually.  And I think
>>>>> one of them has been a general lurking problem w/ spark for a while.
>>>>>
>>>>> 1)  we should ignore disassociation events, as you suggested earlier.
>>>>> They seem to just indicate a temporary problem, and can generally be
>>>>> ignored.  I've found that they're regularly followed by AssociatedEvents,
>>>>> and it seems communication really works fine at that point.
>>>>>
>>>>> 2) Task finished messages get lost.  When this message gets sent, we
>>>>> dont' know it actually gets there:
>>>>>
>>>>>
>>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>>
>>>>> (this is so incredible, I feel I must be overlooking something -- but
>>>>> there is no ack somewhere else that I'm overlooking, is there??)  So, after
>>>>> the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.
>>>>> It hangs b/c the executor has sent some taskFinished messages that never
>>>>> get received by the driver.  So the driver is waiting for some tasks to
>>>>> finish, but the executors think they are all done.
>>>>>
>>>>> I'm gonna add the reliable proxy pattern for this particular
>>>>> interaction and see if its fixes the problem
>>>>>
>>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>>
>>>>> imran
>>>>>
>>>>>
>>>
>>>
>>> --
>>> It's just about how deep your longing is!
>>>
>>>
>>>
>>
>>
>> --
>> It's just about how deep your longing is!
>>
>>

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
thanks everyone for all of the input.

Matei: makes a lot more sense with your explanation of spark's expected
behavior of tcp, I can see why this makes sense now.  But, to show my total
ignorance here, I'm wondering that when the connection does break, are you
sure all of your messages that you thought you sent before the break were
received?  I'm guessing that you don't.  Which is fine, if the response to
that is to have the executor just die completely, and restart.  that was
the behavior I was initially observing with the code on the 2.10 branch,
where the executor handles a DisassociatedEvent explicitly, and dies.

But -- is that the behavior we want?  do we want it to be robust to tcp
connections breaking, without having to completely restart the executor?
you might say that dying & restarting will lead to correct behavior, even
if its inefficient.  But sometimes, I've seen restarts so frequently that
no progress is made.

I don't see why this changed w/ the different versions of akka -- I don't
see any relevant configuration settings that would change how "strongly"
tcp tries to keep the connection alive, but I may be missing something.
But it does seem like the netty configuration options have changed
completely between the two versions:
http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#Remote_Configuration
vs
http://doc.akka.io/docs/akka/2.0.5/scala/remoting.html

btw, akka 2.1.0 also has been built for scala 2.10:
http://search.maven.org/#artifactdetails|com.typesafe.akka|akka-remote_2.10|2.1.0|bundle
and its netty configuration is closer to 2.0.5:
http://doc.akka.io/docs/akka/2.1.0/scala/remoting.html

perhaps someone more knowledge then me about netty & tcp can look through
the changes and decide what the right changes are.

Prashant said:
>Before we conclude something about reliable messaging, I want you to for
once consider other possibilities like >actual network reconnection and may
be a GC pause ? Try connecting something like jconsole (or alike ) and >see
what happens on the driver and executor.
>
>My doubt are since we are using standalone mode where even master and
worker are also actors then if we see >a weird behaviour on the executor
and driver then Why not on master and worker too ? They should also break
>away from each other. For this reason, I am doubting our conclusions and
may be if we narrow down the >problem first before we conclude something.
It is a regression in akka 2.2.3 it uses more memory than it used to >be in
2.1.x.
>See https://github.com/akka/akka/issues/1810


Well, there could easily be the same problem with dropped connections
between master & worker -- they just communicate so little, it doesn't
really matter.  The odds that a message gets dropped between them is very
low, only because there are barely any messages.

I completely agree that the problem could be because of a contention, or gc
pause, etc.  In fact, I'm only giving spark 24 out of 32 cores available on
each box, and 90g out of 125g memory.  I've looked at gc a little with
jstat, and I did see some gc pauses but nothing ridiculous.

But, I think the question remains.  Suppose it is gc pauses, etc. that
cause the disassociation events; what do we do to fix it?  How can we
diagnose the problem, and figure out which of the configuration variables
to tune?  clearly, there *will be* long gc pauses, and the networking layer
needs to be able to deal with them.

still I understand your desire to see if that might be the cause of the
problem in this particular case, so I will dig a little more.


(btw, should I move this thread to the dev list now?  it is getting into
the nitty-gritty of implementation ...)

On Fri, Nov 1, 2013 at 1:15 AM, Matei Zaharia <ma...@gmail.com>wrote:

> Yes, so far they’ve been built on that assumption — not that Akka would
> *guarantee* delivery in that as soon as the send() call returns you know
> it’s delivered, but that Akka would act the same way as a TCP socket,
> allowing you to send a stream of messages in order and hear when the
> connection breaks. Maybe that isn’t what they want to provide, but I'd find
> it weird, because it’s very easy to write a server with this property.
>
> Matei
>
> On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <
> sri.rams85@gmail.com> wrote:
>
> Sorry if I my understanding is wrong. May be, for this particular case it
> might be something to do with the load/network, but, in general, are you
> saying that, we build these communication channels(block manager
> communication, task events communication, etc) assuming akka would take
> care of it? I somehow feel that, it's being overly optimistic. Correct me
> if I am wrong.
>
>
>
> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com>wrote:
>
>> It’s true that Akka’s delivery guarantees are in general at-most-once,
>> but if you look at the text there it says that they differ by transport. In
>> the previous version, I’m quite sure that except maybe in very rare
>> circumstances or cases where we had a bug, Akka’s remote layer always kept
>> connections up between each pair of hosts. So the guarantee was that as
>> long as you haven’t received a “disconnected” event, your messages are
>> being delivered, though of course when you do receive that event you don’t
>> know which messages have really made it through unless you acked them. But
>> that didn’t matter for our use case — from our point of view an executor
>> was either up or down.
>>
>> For this reason I still think it should be possible to configure Akka to
>> do the same on 2.2. Most likely some timeouts just got lower. With large
>> heaps you can easily get a GC pause of 60 seconds, so these timeouts should
>> be in the minutes.
>>
>> If for some reason this isn’t the case, then we have a bigger problem —
>> there are *lots* of messages beyond task-finished that need to be sent
>> reliably, including things like block manager events (a block was added /
>> removed on this node) and commands to tell the block manager to drop data.
>> It would be silly to implement acks at the application level for all these.
>> But I doubt this is the case. Prashant’s observation that the standalone
>> cluster manager stayed up is a further sign that this might be due to GC.
>>
>> Matei
>>
>> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <
>> sri.rams85@gmail.com> wrote:
>>
>> Hi Imran,
>> Just to add, we've noticed dis-associations in a couple projects that we
>> built(using akka 2.2.x not spark). We went to some details to find out what
>> was happening. As Matei, suggested, Akka keeps the TCP connection open and
>> uses that to talk to peers. We noticed that in our case, initially, we were
>> seeing dis-associations generally at the end of keep-alive duration. So,
>> when the keep-alive duration ends, at the TCP layer, a keep-alive probe
>> gets sent to inform the peer on the other side that the connection is still
>> alive/valid. For some reason, the probe dint renew the keep-alive
>> connection and we saw a lot of dis-associations during that time. Later, we
>> realized this was not a pattern either. This thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains the full history of our discussions with the Akka team. It's still
>> open and unclear as to what was causing it for our case.
>> We tried tweaking various settings of akka(wrt heartbeats, failure
>> detector, even plugged-in our own failure detector with no effect).
>>
>> Imran - Just to clarify your point on message delivery - akka's message
>> delivery policy is at-most-once. However, there's no guarantee for a
>> message to be delivered to a peer. The documentation clearly explains that.
>> http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
>> the responsibility of the application developer to handle cases where
>> message is suspected to be not have been delivered.
>>
>> I hope this helps.
>>
>>
>>
>>
>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>>
>>> unfortunately that change wasn't the silver bullet I was hoping for.
>>> Even with
>>> 1) ignoring DisassociatedEvent
>>> 2) executor uses ReliableProxy to send messages back to driver
>>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>>
>>>
>>> there is a lot of weird behavior.  First, there are a few
>>> DisassociatedEvents, but some that are followed by AssociatedEvents, so
>>> that seems ok.  But sometimes the re-associations are immediately followed
>>> by this:
>>>
>>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
>>> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
>>> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
>>> akka.tcp://spark@<driver>:41321] [
>>> akka.remote.InvalidAssociation: Invalid address:
>>> akka.tcp://spark@<driver>:41321
>>> Caused by: akka.remote.transport.Transport$InvalidAssociationException:
>>> The remote system has quarantined this system. No further associations to
>>> the remote system are possible until this system is restarted.
>>> ]
>>>
>>> On the driver, there are messages like:
>>>
>>> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3]
>>> [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now
>>> quarantined, all messages to this address will be delivered to dead letters.
>>> [WARN] [10/31/2013 18:51:10.845]
>>> [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher]
>>> Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>>>
>>>
>>> and when the driver does decide that the executor has been terminated,
>>> it removes the executor, but doesn't start another one.
>>>
>>> there are a ton of messages also about messages to the block manager
>>> master ... I'm wondering if there are other parts of the system that need
>>> to use a reliable proxy (or some sort of acknowledgement).
>>>
>>> I really don't think this was working properly even w/ previous versions
>>> of spark / akka.  I'm still learning about akka, but I think you always
>>> need an ack to be confident w/ remote communicate.  Perhaps the old version
>>> of akka just had more robust defaults or something, but I bet it could
>>> still have the same problems.  Even before, I have seen the driver thinking
>>> there were running tasks, but nothing happening on any executor -- it was
>>> just rare enough (and hard to reproduce) that I never bothered looking into
>>> it more.
>>>
>>> I will keep digging ...
>>>
>>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com>wrote:
>>>
>>>> BTW the problem might be the Akka failure detector settings that seem
>>>> new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>>
>>>> Their timeouts seem pretty aggressive by default — around 10 seconds.
>>>> This can easily be too little if you have large garbage collections. We
>>>> should make sure they are higher than our own node failure detection
>>>> timeouts.
>>>>
>>>> Matei
>>>>
>>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>>>>
>>>> pretty sure I found the problem -- two problems actually.  And I think
>>>> one of them has been a general lurking problem w/ spark for a while.
>>>>
>>>> 1)  we should ignore disassociation events, as you suggested earlier.
>>>> They seem to just indicate a temporary problem, and can generally be
>>>> ignored.  I've found that they're regularly followed by AssociatedEvents,
>>>> and it seems communication really works fine at that point.
>>>>
>>>> 2) Task finished messages get lost.  When this message gets sent, we
>>>> dont' know it actually gets there:
>>>>
>>>>
>>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>>
>>>> (this is so incredible, I feel I must be overlooking something -- but
>>>> there is no ack somewhere else that I'm overlooking, is there??)  So, after
>>>> the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.
>>>> It hangs b/c the executor has sent some taskFinished messages that never
>>>> get received by the driver.  So the driver is waiting for some tasks to
>>>> finish, but the executors think they are all done.
>>>>
>>>> I'm gonna add the reliable proxy pattern for this particular
>>>> interaction and see if its fixes the problem
>>>>
>>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>>
>>>> imran
>>>>
>>>>
>>
>>
>> --
>> It's just about how deep your longing is!
>>
>>
>>
>
>
> --
> It's just about how deep your longing is!
>
>

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
Yes, so far they’ve been built on that assumption — not that Akka would *guarantee* delivery in that as soon as the send() call returns you know it’s delivered, but that Akka would act the same way as a TCP socket, allowing you to send a stream of messages in order and hear when the connection breaks. Maybe that isn’t what they want to provide, but I'd find it weird, because it’s very easy to write a server with this property.

Matei

On Oct 31, 2013, at 9:58 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:

> Sorry if I my understanding is wrong. May be, for this particular case it might be something to do with the load/network, but, in general, are you saying that, we build these communication channels(block manager communication, task events communication, etc) assuming akka would take care of it? I somehow feel that, it's being overly optimistic. Correct me if I am wrong.
> 
> 
> 
> On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com> wrote:
> It’s true that Akka’s delivery guarantees are in general at-most-once, but if you look at the text there it says that they differ by transport. In the previous version, I’m quite sure that except maybe in very rare circumstances or cases where we had a bug, Akka’s remote layer always kept connections up between each pair of hosts. So the guarantee was that as long as you haven’t received a “disconnected” event, your messages are being delivered, though of course when you do receive that event you don’t know which messages have really made it through unless you acked them. But that didn’t matter for our use case — from our point of view an executor was either up or down.
> 
> For this reason I still think it should be possible to configure Akka to do the same on 2.2. Most likely some timeouts just got lower. With large heaps you can easily get a GC pause of 60 seconds, so these timeouts should be in the minutes.
> 
> If for some reason this isn’t the case, then we have a bigger problem — there are *lots* of messages beyond task-finished that need to be sent reliably, including things like block manager events (a block was added / removed on this node) and commands to tell the block manager to drop data. It would be silly to implement acks at the application level for all these. But I doubt this is the case. Prashant’s observation that the standalone cluster manager stayed up is a further sign that this might be due to GC.
> 
> Matei
> 
> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:
> 
>> Hi Imran,
>> Just to add, we've noticed dis-associations in a couple projects that we built(using akka 2.2.x not spark). We went to some details to find out what was happening. As Matei, suggested, Akka keeps the TCP connection open and uses that to talk to peers. We noticed that in our case, initially, we were seeing dis-associations generally at the end of keep-alive duration. So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe gets sent to inform the peer on the other side that the connection is still alive/valid. For some reason, the probe dint renew the keep-alive connection and we saw a lot of dis-associations during that time. Later, we realized this was not a pattern either. This thread contains the full history of our discussions with the Akka team. It's still open and unclear as to what was causing it for our case. 
>> We tried tweaking various settings of akka(wrt heartbeats, failure detector, even plugged-in our own failure detector with no effect).
>> 
>> Imran - Just to clarify your point on message delivery - akka's message delivery policy is at-most-once. However, there's no guarantee for a message to be delivered to a peer. The documentation clearly explains that. http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's the responsibility of the application developer to handle cases where message is suspected to be not have been delivered. 
>> 
>> I hope this helps.
>> 
>> 
>> 
>> 
>> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:
>> 
>> unfortunately that change wasn't the silver bullet I was hoping for.  Even with
>> 1) ignoring DisassociatedEvent
>> 2) executor uses ReliableProxy to send messages back to driver
>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>> 
>> 
>> there is a lot of weird behavior.  First, there are a few DisassociatedEvents, but some that are followed by AssociatedEvents, so that seems ok.  But sometimes the re-associations are immediately followed by this:
>> 
>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441] -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address: akka.tcp://spark@<driver>:41321] [
>> akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@<driver>:41321
>> Caused by: akka.remote.transport.Transport$InvalidAssociationException: The remote system has quarantined this system. No further associations to the remote system are possible until this system is restarted.
>> ]
>> 
>> On the driver, there are messages like:
>> 
>> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3] [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all messages to this address will be delivered to dead letters.
>> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher] Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
>> 
>> 
>> and when the driver does decide that the executor has been terminated, it removes the executor, but doesn't start another one.
>> 
>> there are a ton of messages also about messages to the block manager master ... I'm wondering if there are other parts of the system that need to use a reliable proxy (or some sort of acknowledgement).
>> 
>> I really don't think this was working properly even w/ previous versions of spark / akka.  I'm still learning about akka, but I think you always need an ack to be confident w/ remote communicate.  Perhaps the old version of akka just had more robust defaults or something, but I bet it could still have the same problems.  Even before, I have seen the driver thinking there were running tasks, but nothing happening on any executor -- it was just rare enough (and hard to reproduce) that I never bothered looking into it more.
>> 
>> I will keep digging ...
>> 
>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com> wrote:
>> BTW the problem might be the Akka failure detector settings that seem new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>> 
>> Their timeouts seem pretty aggressive by default — around 10 seconds. This can easily be too little if you have large garbage collections. We should make sure they are higher than our own node failure detection timeouts.
>> 
>> Matei
>> 
>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>> 
>>> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
>>> 
>>> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
>>> 
>>> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
>>> 
>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>> 
>>> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
>>> 
>>> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>> 
>>> imran
>> 
>> 
>> 
>> -- 
>> It's just about how deep your longing is!
> 
> 
> 
> 
> -- 
> It's just about how deep your longing is!


Re: executor failures w/ scala 2.10

Posted by Sriram Ramachandrasekaran <sr...@gmail.com>.
Sorry if I my understanding is wrong. May be, for this particular case it
might be something to do with the load/network, but, in general, are you
saying that, we build these communication channels(block manager
communication, task events communication, etc) assuming akka would take
care of it? I somehow feel that, it's being overly optimistic. Correct me
if I am wrong.



On Fri, Nov 1, 2013 at 10:08 AM, Matei Zaharia <ma...@gmail.com>wrote:

> It’s true that Akka’s delivery guarantees are in general at-most-once, but
> if you look at the text there it says that they differ by transport. In the
> previous version, I’m quite sure that except maybe in very rare
> circumstances or cases where we had a bug, Akka’s remote layer always kept
> connections up between each pair of hosts. So the guarantee was that as
> long as you haven’t received a “disconnected” event, your messages are
> being delivered, though of course when you do receive that event you don’t
> know which messages have really made it through unless you acked them. But
> that didn’t matter for our use case — from our point of view an executor
> was either up or down.
>
> For this reason I still think it should be possible to configure Akka to
> do the same on 2.2. Most likely some timeouts just got lower. With large
> heaps you can easily get a GC pause of 60 seconds, so these timeouts should
> be in the minutes.
>
> If for some reason this isn’t the case, then we have a bigger problem —
> there are *lots* of messages beyond task-finished that need to be sent
> reliably, including things like block manager events (a block was added /
> removed on this node) and commands to tell the block manager to drop data.
> It would be silly to implement acks at the application level for all these.
> But I doubt this is the case. Prashant’s observation that the standalone
> cluster manager stayed up is a further sign that this might be due to GC.
>
> Matei
>
> On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <
> sri.rams85@gmail.com> wrote:
>
> Hi Imran,
> Just to add, we've noticed dis-associations in a couple projects that we
> built(using akka 2.2.x not spark). We went to some details to find out what
> was happening. As Matei, suggested, Akka keeps the TCP connection open and
> uses that to talk to peers. We noticed that in our case, initially, we were
> seeing dis-associations generally at the end of keep-alive duration. So,
> when the keep-alive duration ends, at the TCP layer, a keep-alive probe
> gets sent to inform the peer on the other side that the connection is still
> alive/valid. For some reason, the probe dint renew the keep-alive
> connection and we saw a lot of dis-associations during that time. Later, we
> realized this was not a pattern either. This thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains the full history of our discussions with the Akka team. It's still
> open and unclear as to what was causing it for our case.
> We tried tweaking various settings of akka(wrt heartbeats, failure
> detector, even plugged-in our own failure detector with no effect).
>
> Imran - Just to clarify your point on message delivery - akka's message
> delivery policy is at-most-once. However, there's no guarantee for a
> message to be delivered to a peer. The documentation clearly explains that.
> http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
> the responsibility of the application developer to handle cases where
> message is suspected to be not have been delivered.
>
> I hope this helps.
>
>
>
>
> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:
>
>>
>> unfortunately that change wasn't the silver bullet I was hoping for.
>> Even with
>> 1) ignoring DisassociatedEvent
>> 2) executor uses ReliableProxy to send messages back to driver
>> 3) turn up akka.remote.watch-failure-detector.threshold=12
>>
>>
>> there is a lot of weird behavior.  First, there are a few
>> DisassociatedEvents, but some that are followed by AssociatedEvents, so
>> that seems ok.  But sometimes the re-associations are immediately followed
>> by this:
>>
>> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
>> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
>> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
>> akka.tcp://spark@<driver>:41321] [
>> akka.remote.InvalidAssociation: Invalid address:
>> akka.tcp://spark@<driver>:41321
>> Caused by: akka.remote.transport.Transport$InvalidAssociationException:
>> The remote system has quarantined this system. No further associations to
>> the remote system are possible until this system is restarted.
>> ]
>>
>> On the driver, there are messages like:
>>
>> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3]
>> [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now
>> quarantined, all messages to this address will be delivered to dead letters.
>> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20]
>> [akka://spark/system/remote-watcher] Detected unreachable:
>> [akka.tcp://sparkExecutor@<executor>:41441]
>>
>>
>> and when the driver does decide that the executor has been terminated, it
>> removes the executor, but doesn't start another one.
>>
>> there are a ton of messages also about messages to the block manager
>> master ... I'm wondering if there are other parts of the system that need
>> to use a reliable proxy (or some sort of acknowledgement).
>>
>> I really don't think this was working properly even w/ previous versions
>> of spark / akka.  I'm still learning about akka, but I think you always
>> need an ack to be confident w/ remote communicate.  Perhaps the old version
>> of akka just had more robust defaults or something, but I bet it could
>> still have the same problems.  Even before, I have seen the driver thinking
>> there were running tasks, but nothing happening on any executor -- it was
>> just rare enough (and hard to reproduce) that I never bothered looking into
>> it more.
>>
>> I will keep digging ...
>>
>> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com>wrote:
>>
>>> BTW the problem might be the Akka failure detector settings that seem
>>> new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>>
>>> Their timeouts seem pretty aggressive by default — around 10 seconds.
>>> This can easily be too little if you have large garbage collections. We
>>> should make sure they are higher than our own node failure detection
>>> timeouts.
>>>
>>> Matei
>>>
>>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>>>
>>> pretty sure I found the problem -- two problems actually.  And I think
>>> one of them has been a general lurking problem w/ spark for a while.
>>>
>>> 1)  we should ignore disassociation events, as you suggested earlier.
>>> They seem to just indicate a temporary problem, and can generally be
>>> ignored.  I've found that they're regularly followed by AssociatedEvents,
>>> and it seems communication really works fine at that point.
>>>
>>> 2) Task finished messages get lost.  When this message gets sent, we
>>> dont' know it actually gets there:
>>>
>>>
>>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>>
>>> (this is so incredible, I feel I must be overlooking something -- but
>>> there is no ack somewhere else that I'm overlooking, is there??)  So, after
>>> the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.
>>> It hangs b/c the executor has sent some taskFinished messages that never
>>> get received by the driver.  So the driver is waiting for some tasks to
>>> finish, but the executors think they are all done.
>>>
>>> I'm gonna add the reliable proxy pattern for this particular interaction
>>> and see if its fixes the problem
>>>
>>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>>
>>> imran
>>>
>>>
>
>
> --
> It's just about how deep your longing is!
>
>
>


-- 
It's just about how deep your longing is!

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
It’s true that Akka’s delivery guarantees are in general at-most-once, but if you look at the text there it says that they differ by transport. In the previous version, I’m quite sure that except maybe in very rare circumstances or cases where we had a bug, Akka’s remote layer always kept connections up between each pair of hosts. So the guarantee was that as long as you haven’t received a “disconnected” event, your messages are being delivered, though of course when you do receive that event you don’t know which messages have really made it through unless you acked them. But that didn’t matter for our use case — from our point of view an executor was either up or down.

For this reason I still think it should be possible to configure Akka to do the same on 2.2. Most likely some timeouts just got lower. With large heaps you can easily get a GC pause of 60 seconds, so these timeouts should be in the minutes.

If for some reason this isn’t the case, then we have a bigger problem — there are *lots* of messages beyond task-finished that need to be sent reliably, including things like block manager events (a block was added / removed on this node) and commands to tell the block manager to drop data. It would be silly to implement acks at the application level for all these. But I doubt this is the case. Prashant’s observation that the standalone cluster manager stayed up is a further sign that this might be due to GC.

Matei

On Oct 31, 2013, at 9:11 PM, Sriram Ramachandrasekaran <sr...@gmail.com> wrote:

> Hi Imran,
> Just to add, we've noticed dis-associations in a couple projects that we built(using akka 2.2.x not spark). We went to some details to find out what was happening. As Matei, suggested, Akka keeps the TCP connection open and uses that to talk to peers. We noticed that in our case, initially, we were seeing dis-associations generally at the end of keep-alive duration. So, when the keep-alive duration ends, at the TCP layer, a keep-alive probe gets sent to inform the peer on the other side that the connection is still alive/valid. For some reason, the probe dint renew the keep-alive connection and we saw a lot of dis-associations during that time. Later, we realized this was not a pattern either. This thread contains the full history of our discussions with the Akka team. It's still open and unclear as to what was causing it for our case. 
> We tried tweaking various settings of akka(wrt heartbeats, failure detector, even plugged-in our own failure detector with no effect).
> 
> Imran - Just to clarify your point on message delivery - akka's message delivery policy is at-most-once. However, there's no guarantee for a message to be delivered to a peer. The documentation clearly explains that. http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's the responsibility of the application developer to handle cases where message is suspected to be not have been delivered. 
> 
> I hope this helps.
> 
> 
> 
> 
> On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:
> 
> unfortunately that change wasn't the silver bullet I was hoping for.  Even with
> 1) ignoring DisassociatedEvent
> 2) executor uses ReliableProxy to send messages back to driver
> 3) turn up akka.remote.watch-failure-detector.threshold=12
> 
> 
> there is a lot of weird behavior.  First, there are a few DisassociatedEvents, but some that are followed by AssociatedEvents, so that seems ok.  But sometimes the re-associations are immediately followed by this:
> 
> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441] -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address: akka.tcp://spark@<driver>:41321] [
> akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@<driver>:41321
> Caused by: akka.remote.transport.Transport$InvalidAssociationException: The remote system has quarantined this system. No further associations to the remote system are possible until this system is restarted.
> ]
> 
> On the driver, there are messages like:
> 
> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3] [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now quarantined, all messages to this address will be delivered to dead letters.
> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20] [akka://spark/system/remote-watcher] Detected unreachable: [akka.tcp://sparkExecutor@<executor>:41441]
> 
> 
> and when the driver does decide that the executor has been terminated, it removes the executor, but doesn't start another one.
> 
> there are a ton of messages also about messages to the block manager master ... I'm wondering if there are other parts of the system that need to use a reliable proxy (or some sort of acknowledgement).
> 
> I really don't think this was working properly even w/ previous versions of spark / akka.  I'm still learning about akka, but I think you always need an ack to be confident w/ remote communicate.  Perhaps the old version of akka just had more robust defaults or something, but I bet it could still have the same problems.  Even before, I have seen the driver thinking there were running tasks, but nothing happening on any executor -- it was just rare enough (and hard to reproduce) that I never bothered looking into it more.
> 
> I will keep digging ...
> 
> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com> wrote:
> BTW the problem might be the Akka failure detector settings that seem new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
> 
> Their timeouts seem pretty aggressive by default — around 10 seconds. This can easily be too little if you have large garbage collections. We should make sure they are higher than our own node failure detection timeouts.
> 
> Matei
> 
> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
> 
>> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
>> 
>> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
>> 
>> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
>> 
>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>> 
>> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
>> 
>> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>> 
>> imran
> 
> 
> 
> -- 
> It's just about how deep your longing is!


Re: executor failures w/ scala 2.10

Posted by Sriram Ramachandrasekaran <sr...@gmail.com>.
Hi Imran,
Just to add, we've noticed dis-associations in a couple projects that we
built(using akka 2.2.x not spark). We went to some details to find out what
was happening. As Matei, suggested, Akka keeps the TCP connection open and
uses that to talk to peers. We noticed that in our case, initially, we were
seeing dis-associations generally at the end of keep-alive duration. So,
when the keep-alive duration ends, at the TCP layer, a keep-alive probe
gets sent to inform the peer on the other side that the connection is still
alive/valid. For some reason, the probe dint renew the keep-alive
connection and we saw a lot of dis-associations during that time. Later, we
realized this was not a pattern either. This
thread<https://groups.google.com/forum/#!msg/akka-user/RYxaPl_nby4/1USHDFIRgOkJ>contains
the full history of our discussions with the Akka team. It's still
open and unclear as to what was causing it for our case.
We tried tweaking various settings of akka(wrt heartbeats, failure
detector, even plugged-in our own failure detector with no effect).

Imran - Just to clarify your point on message delivery - akka's message
delivery policy is at-most-once. However, there's no guarantee for a
message to be delivered to a peer. The documentation clearly explains that.
http://doc.akka.io/docs/akka/2.0.2/general/message-send-semantics.html. It's
the responsibility of the application developer to handle cases where
message is suspected to be not have been delivered.

I hope this helps.




On Fri, Nov 1, 2013 at 8:35 AM, Imran Rashid <im...@quantifind.com> wrote:

>
> unfortunately that change wasn't the silver bullet I was hoping for.  Even
> with
> 1) ignoring DisassociatedEvent
> 2) executor uses ReliableProxy to send messages back to driver
> 3) turn up akka.remote.watch-failure-detector.threshold=12
>
>
> there is a lot of weird behavior.  First, there are a few
> DisassociatedEvents, but some that are followed by AssociatedEvents, so
> that seems ok.  But sometimes the re-associations are immediately followed
> by this:
>
> 13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
> lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
> -> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
> akka.tcp://spark@<driver>:41321] [
> akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@
> <driver>:41321
> Caused by: akka.remote.transport.Transport$InvalidAssociationException:
> The remote system has quarantined this system. No further associations to
> the remote system are possible until this system is restarted.
> ]
>
> On the driver, there are messages like:
>
> [INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3]
> [Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now
> quarantined, all messages to this address will be delivered to dead letters.
> [WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20]
> [akka://spark/system/remote-watcher] Detected unreachable:
> [akka.tcp://sparkExecutor@<executor>:41441]
>
>
> and when the driver does decide that the executor has been terminated, it
> removes the executor, but doesn't start another one.
>
> there are a ton of messages also about messages to the block manager
> master ... I'm wondering if there are other parts of the system that need
> to use a reliable proxy (or some sort of acknowledgement).
>
> I really don't think this was working properly even w/ previous versions
> of spark / akka.  I'm still learning about akka, but I think you always
> need an ack to be confident w/ remote communicate.  Perhaps the old version
> of akka just had more robust defaults or something, but I bet it could
> still have the same problems.  Even before, I have seen the driver thinking
> there were running tasks, but nothing happening on any executor -- it was
> just rare enough (and hard to reproduce) that I never bothered looking into
> it more.
>
> I will keep digging ...
>
> On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com>wrote:
>
>> BTW the problem might be the Akka failure detector settings that seem new
>> in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>>
>> Their timeouts seem pretty aggressive by default — around 10 seconds.
>> This can easily be too little if you have large garbage collections. We
>> should make sure they are higher than our own node failure detection
>> timeouts.
>>
>> Matei
>>
>> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>>
>> pretty sure I found the problem -- two problems actually.  And I think
>> one of them has been a general lurking problem w/ spark for a while.
>>
>> 1)  we should ignore disassociation events, as you suggested earlier.
>> They seem to just indicate a temporary problem, and can generally be
>> ignored.  I've found that they're regularly followed by AssociatedEvents,
>> and it seems communication really works fine at that point.
>>
>> 2) Task finished messages get lost.  When this message gets sent, we
>> dont' know it actually gets there:
>>
>>
>> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>>
>> (this is so incredible, I feel I must be overlooking something -- but
>> there is no ack somewhere else that I'm overlooking, is there??)  So, after
>> the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.
>> It hangs b/c the executor has sent some taskFinished messages that never
>> get received by the driver.  So the driver is waiting for some tasks to
>> finish, but the executors think they are all done.
>>
>> I'm gonna add the reliable proxy pattern for this particular interaction
>> and see if its fixes the problem
>>
>> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>>
>> imran
>>
>>


-- 
It's just about how deep your longing is!

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
unfortunately that change wasn't the silver bullet I was hoping for.  Even
with
1) ignoring DisassociatedEvent
2) executor uses ReliableProxy to send messages back to driver
3) turn up akka.remote.watch-failure-detector.threshold=12


there is a lot of weird behavior.  First, there are a few
DisassociatedEvents, but some that are followed by AssociatedEvents, so
that seems ok.  But sometimes the re-associations are immediately followed
by this:

13/10/31 18:51:10 INFO executor.StandaloneExecutorBackend: got
lifecycleevent: AssociationError [akka.tcp://sparkExecutor@<executor>:41441]
-> [akka.tcp://spark@<driver>:41321]: Error [Invalid address:
akka.tcp://spark@<driver>:41321] [
akka.remote.InvalidAssociation: Invalid address: akka.tcp://spark@
<driver>:41321
Caused by: akka.remote.transport.Transport$InvalidAssociationException: The
remote system has quarantined this system. No further associations to the
remote system are possible until this system is restarted.
]

On the driver, there are messages like:

[INFO] [10/31/2013 18:51:07.838] [spark-akka.actor.default-dispatcher-3]
[Remoting] Address [akka.tcp://sparkExecutor@<executor>:46123] is now
quarantined, all messages to this address will be delivered to dead letters.
[WARN] [10/31/2013 18:51:10.845] [spark-akka.actor.default-dispatcher-20]
[akka://spark/system/remote-watcher] Detected unreachable:
[akka.tcp://sparkExecutor@<executor>:41441]


and when the driver does decide that the executor has been terminated, it
removes the executor, but doesn't start another one.

there are a ton of messages also about messages to the block manager master
... I'm wondering if there are other parts of the system that need to use a
reliable proxy (or some sort of acknowledgement).

I really don't think this was working properly even w/ previous versions of
spark / akka.  I'm still learning about akka, but I think you always need
an ack to be confident w/ remote communicate.  Perhaps the old version of
akka just had more robust defaults or something, but I bet it could still
have the same problems.  Even before, I have seen the driver thinking there
were running tasks, but nothing happening on any executor -- it was just
rare enough (and hard to reproduce) that I never bothered looking into it
more.

I will keep digging ...

On Thu, Oct 31, 2013 at 4:36 PM, Matei Zaharia <ma...@gmail.com>wrote:

> BTW the problem might be the Akka failure detector settings that seem new
> in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html
>
> Their timeouts seem pretty aggressive by default — around 10 seconds. This
> can easily be too little if you have large garbage collections. We should
> make sure they are higher than our own node failure detection timeouts.
>
> Matei
>
> On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:
>
> pretty sure I found the problem -- two problems actually.  And I think one
> of them has been a general lurking problem w/ spark for a while.
>
> 1)  we should ignore disassociation events, as you suggested earlier.
> They seem to just indicate a temporary problem, and can generally be
> ignored.  I've found that they're regularly followed by AssociatedEvents,
> and it seems communication really works fine at that point.
>
> 2) Task finished messages get lost.  When this message gets sent, we dont'
> know it actually gets there:
>
>
> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
>
> (this is so incredible, I feel I must be overlooking something -- but
> there is no ack somewhere else that I'm overlooking, is there??)  So, after
> the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.
> It hangs b/c the executor has sent some taskFinished messages that never
> get received by the driver.  So the driver is waiting for some tasks to
> finish, but the executors think they are all done.
>
> I'm gonna add the reliable proxy pattern for this particular interaction
> and see if its fixes the problem
>
> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
>
> imran
>
>

Re: executor failures w/ scala 2.10

Posted by Matei Zaharia <ma...@gmail.com>.
BTW the problem might be the Akka failure detector settings that seem new in 2.2: http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html

Their timeouts seem pretty aggressive by default — around 10 seconds. This can easily be too little if you have large garbage collections. We should make sure they are higher than our own node failure detection timeouts.

Matei

On Oct 31, 2013, at 1:33 PM, Imran Rashid <im...@quantifind.com> wrote:

> pretty sure I found the problem -- two problems actually.  And I think one of them has been a general lurking problem w/ spark for a while.
> 
> 1)  we should ignore disassociation events, as you suggested earlier.  They seem to just indicate a temporary problem, and can generally be ignored.  I've found that they're regularly followed by AssociatedEvents, and it seems communication really works fine at that point.
> 
> 2) Task finished messages get lost.  When this message gets sent, we dont' know it actually gets there:
> 
> https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90
> 
> (this is so incredible, I feel I must be overlooking something -- but there is no ack somewhere else that I'm overlooking, is there??)  So, after the patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It hangs b/c the executor has sent some taskFinished messages that never get received by the driver.  So the driver is waiting for some tasks to finish, but the executors think they are all done.
> 
> I'm gonna add the reliable proxy pattern for this particular interaction and see if its fixes the problem
> http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy
> 
> imran
> 
> 
> 
> On Thu, Oct 31, 2013 at 1:17 PM, Imran Rashid <im...@quantifind.com> wrote:
> Hi Prashant,
> 
> thanks for looking into this.  I don't have any answers yet, but just wanted to send you an update.  I finally figured out how to get all the akka logging turned on, so I'm looking at those for more info.  One thing immediately jumped out at me -- the Disassociation is actually immediatley followed by an Association!  so maybe I came to the wrong conclusion of our test of ignoring the DisassociatedEvent.  I'm going to try it again -- hopefully w/ the logging on, I can find out more about what is going on.  I might ask on akka list for help w/ what to look for.  also this thread makes me think that it really should just re-associate:
> https://groups.google.com/forum/#!searchin/akka-user/Disassociated/akka-user/SajwwbyTriQ/8oxjbZtawxoJ
> 
> also, I've noticed that actually these failures are *not* more likely the longer the job runs (or not that much, anyway).  when I reorder our jobs, I can get the failures pretty regularly within 10 minutes.  It seems to be more of a function of what the jobs do.  I'll also see if I can produce a simple example which exhibits the same problem.
> 
> thanks!
> 
> Imran
> 
> 
> 
> On Thu, Oct 31, 2013 at 1:37 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Hi Imran,
> 
> So now my jobs are running for more than 18 hours and there is not a single executor failure, I can keep it running for more but I don't think that is going to help. So either you have to give me a way to reproduce the issue (Which would be great !! ) or you have to diagnose for yourself what exactly is causing it, a disconnection usually result in Disassociation event on which we are terminating the executor. In a normal operation this usually means that driver has finished and those log messages are totally fine for that matter. I am not sure why you would not see such a thing in 0.8 release and in scala 2.10, can't say anything until I can see what is happening. 
> 
> 
> 
> On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Can you show us the sample job ? do you do sc.stop at the end or System.exit ? Try sc.stop too..
> 
> 
> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com> wrote:
> yeah, just causes them to hang.
> 
> the first "deadLetters" message shows up about the same time.  Oddly, after it first happens, I keep getting some results trickling in from those executors.  (maybe they were just queued up on the driver already, I dunno.)  but then it just hangs.  the stage has a few more tasks to be run, but the executors are just idle, they're not running anything.
> 
> I'm gonna try manually listening for more Association events listed here & logging them
> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
> 
> imran
> 
> 
> 
> 
> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <sc...@gmail.com> wrote:
> I am guessing something wrong with using Dissociation event then.
> 
> Try applying something on the lines of this patch. This might cause the executors to hang so be prepared for that. 
> 
> diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> index 4e8052a..1ec5d19 100644
> --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>          executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
>        }
>  
> -    case DisassociatedEvent(_, _, _) =>
> -      logError("Driver terminated or disconnected! Shutting down.")
> +    case Terminated(actor) =>
> +      logError("Driver terminated Shutting down.")
>        System.exit(1)
> +
> +    // case DisassociatedEvent(_, _, _) =>
> +    //   logError("Driver terminated or disconnected! Shutting down.")
> +    //   System.exit(1)
>    }
>  
>    override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) {
> diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> index b6f0ec9..9955484 100644
> --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
>        case Terminated(actor) =>
>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
>  
> -      case DisassociatedEvent(_, remoteAddress, _) =>
> -        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected"))
> +      // case DisassociatedEvent(_, remoteAddress, _) =>
> +      //   addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected"))
>  
>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>          addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown"))
> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
>      // Remove a disconnected slave from the cluster
>      def removeExecutor(executorId: String, reason: String) {
>        if (executorActor.contains(executorId)) {
> -        logInfo("Executor " + executorId + " disconnected, so removing it")
> +        logInfo("Executor " + executorId + " disconnected, so removing it, reason:" + reason)
>          val numCores = freeCores(executorId)
>          actorToExecutorId -= executorActor(executorId)
>          addressToExecutorId -= executorAddress(executorId)
> 
> 
> 
> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com> wrote:
> ok, so I applied a few patches
> 
> https://github.com/quantifind/incubator-spark/pull/1/files
> 
> and ran it again, with these options:
> 
> -Dspark.akka.stdout-loglevel=DEBUG \
>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>   -Dspark.akkaExtra.akka.actor.debug.receive=on \  -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>   -Dspark.akkaExtra.akka.log-config-on-start=on
> 
> On the driver, I see:
> 
> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141] was not delivered. [1] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO  org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1 disconnected, so removing it, reason:remote Akka client disconnected
> 
> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on dhd2.quantifind.com: remote Akka client disconnected
> 
> 
> on the worker, stderr:
> 
> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
> 
> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver terminated or disconnected! Shutting down.Disassociated [akka.tcp://sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://spark@ddd0.quantifind.com:36730]
> 
> and unfortunately, all those akka debug options give me *no* useful info in the worker stdout:
> 
> Starting akka system "sparkExecutor" using config:
> 
>       akka.daemonic = on
>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>       akka.stdout-loglevel = "DEBUG"
>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>       akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>       akka.remote.netty.tcp.port = 0
>       akka.remote.netty.tcp.connection-timeout = 60 s
>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>       akka.remote.netty.tcp.execution-pool-size = 4
>       akka.actor.default-dispatcher.throughput = 15
>       akka.remote.log-remote-lifecycle-events = off
>                        akka.remote.log-sent-messages = on
> akka.remote.log-received-messages = on
> akka.logLevel = DEBUG
> akka.actor.debug.autoreceive = on
> akka.actor.debug.lifecycle = on
> akka.actor.debug.receive = on
> akka.log-config-on-start = on
> akka.remote.quarantine-systems-for = off
> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream] StandardOutLogger started
> [DEBUG] [10/30/2013 08:40:30.438] [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
> [DEBUG] [10/30/2013 08:40:30.446] [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user] started (akka.actor.LocalActorRefProvider$Guardian@72608760)
> [DEBUG] [10/30/2013 08:40:30.447] [sparkExecutor-akka.actor.default-dispatcher-4] [akka://sparkExecutor/system] started (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
> [DEBUG] [10/30/2013 08:40:30.454] [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now supervising Actor[akka://sparkExecutor/user]
> [DEBUG] [10/30/2013 08:40:30.454] [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now supervising Actor[akka://sparkExecutor/system]
> [DEBUG] [10/30/2013 08:40:30.468] [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user] now monitoring Actor[akka://sparkExecutor/system]
> [DEBUG] [10/30/2013 08:40:30.468] [sparkExecutor-akka.actor.default-dispatcher-4] [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
> [DEBUG] [10/30/2013 08:40:30.476] [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/system/log1-Slf4jLogger] started (akka.event.slf4j.Slf4jLogger@24988707)
> [DEBUG] [10/30/2013 08:40:30.477] [sparkExecutor-akka.actor.default-dispatcher-4] [akka://sparkExecutor/system] now supervising Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
> 
> (followed by similar mesages for the "spark" system)
> 
> I dunno if this means much more to you, but it seems to me that for some reason the executor decides to disconnect from the master -- unfortunately we don't know why.  I think my logging configuration is not getting applied correctly, or "log-sent-messages" & "log-received-messages" don't do what I think they do ... something conflicting must be turing that logging off.  There are a zillion different remoting settings:
> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
> 
> I feel like I really need to get those messages on why it disconnected to know which ones to play with.  Any ideas for config changes to see those messages?
> 
> thanks
> 
> 
> 
> 
> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Can you apply this patch too and check the logs of Driver and worker.
> 
> diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> index b6f0ec9..ad0ebf7 100644
> --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
>      // Remove a disconnected slave from the cluster
>      def removeExecutor(executorId: String, reason: String) {
>        if (executorActor.contains(executorId)) {
> -        logInfo("Executor " + executorId + " disconnected, so removing it")
> +        logInfo("Executor " + executorId + " disconnected, so removing it, reason:" + reason)
>          val numCores = freeCores(executorId)
>          actorToExecutorId -= executorActor(executorId)
>          addressToExecutorId -= executorAddress(executorId)
> 
> 
> 
> 
> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com> wrote:
> I just realized something about the failing stages -- they generally occur in steps like this:
> 
> rdd.mapPartitions{itr =>
>   val myCounters = initializeSomeDataStructure()
>   itr.foreach{
>     //update myCounter in here
>     ...
>   }
>   
>   myCounters.iterator.map{ 
>     //some other transformation here ...
>   }
> }
> 
> that is, as a partition is processed, nothing gets output, we just accumulate some values.  Only at the end of the partition do we output some accumulated values.
> 
> These stages don't always fail, and generally they do succeed after the executor has died and a new one has started -- so I'm pretty confident its not a problem w/ the code.  But maybe we need to add something like a periodic heartbeat in this kind of operation?
> 
> 
> 
> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com> wrote:
> I'm gonna try turning on more akka debugging msgs as described at
> http://akka.io/faq/
> and
> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
> 
> unfortunately that will require a patch to spark, but hopefully that will give us more info to go on ...
> 
> 
> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com> wrote:
> I have things running (from scala 2.10 branch) for over 3-4 hours now without a problem and my jobs write data about the same as you suggested. My cluster size is 7 nodes and not *congested* for memory. I going to leave jobs running all night long. Meanwhile I had encourage you to try to spot the problem such that it is reproducible that can help a ton in fixing the issue. 
> 
> Thanks for testing and reporting your experience. I still feel there is something else wrong !. About tolerance for network connection timeouts, setting those properties should work, but I am afraid about Disassociation Event though. I will have to check this is indeed hard to reproduce bug if it is, I mean how do I simulate network delays ?
> 
> 
> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com> wrote:
> This is a spark-standalone setup (not mesos), on our own cluster.
> 
> At first I thought it must be some temporary network problem too -- but the times between receiving task completion events from an executor and declaring it failed are really small, so I didn't think that could possibly be it.  Plus we tried increasing various akka timeouts, but that didn't help.  Or maybe there are some other spark / akka properities we should be setting?  It certainly should be resilient to such a temporary network issue, if that is the problem.
> 
> btw, I think I've noticed this happens most often during ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB total for the entire stage).
> 
> thanks
> 
> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Are you using mesos ? I admit to have not properly tested things on mesos though. 
> 
> 
> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <sc...@gmail.com> wrote:
> Those log messages are new to the Akka 2.2 and are usually seen when a node is disassociated with other by either a network failure or even clean shutdown. This suggests some network issue to me, are you running on EC2 ? It might be a temporary thing in that case. 
> 
> I had like to have more details on the long jobs though, how long ? 
> 
> 
> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com> wrote:
> We've been testing out the 2.10 branch of spark, and we're running into some issues were akka disconnects from the executors after a while.  We ran some simple tests first, and all was well, so we started upgrading our whole codebase to 2.10.  Everything seemed to be working, but then we noticed that when we run long jobs, and then things start failing.
> 
> 
> The first suspicious thing is that we get akka warnings about undeliverable messages sent to deadLetters:
> 
> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700] was not delivered. [4] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700] was not delivered. [5] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 
> 
> Generally within a few seconds after the first such message, there are a bunch more, and then the executor is marked as failed, and a new one is started:
> 
> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO  akka.actor.LocalActorRef - Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://spark/deadLetters] to Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10] dead letters encountered, no more dead letters will be logged. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated: app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
> 
> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added: app-20131029110000-0000/2 on worker-20131029105824-dhd2.quantifind.com-51544 (dhd2.quantifind.com:51544) with 24 cores
> 
> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068] -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
> akka.remote.EndpointAssociationException: Association failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
> Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
> 
> 
> 
> Looking in the logs of the failed executor, there are some similar messages about undeliverable messages, but I don't see any reason:
> 
> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transport.AssociationHandle$Disassociated] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver terminated or disconnected! Shutting down.
> 
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://sparkExecutor/deadLetters] to Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
> 
> 
> After this happens, spark does launch a new executor successfully, and continue the job.  Sometimes, the job just continues happily and there aren't any other problems.  However, that executor may have to run a bunch of steps to re-compute some cached RDDs -- and during that time, another executor may crash similarly, and then we end up in a never ending loop, of one executor crashing, then trying to reload data, while the others sit around.
> 
> I have no idea what is triggering this behavior -- there isn't any particular point in the job that it regularly occurs at.  Certain steps seem more prone to this, but there isn't any step which regularly causes the problem.  In a long pipeline of steps, though, that loop becomes very likely.  I don't think its a timeout issue -- the initial failing executors can be actively completing stages just seconds before this failure happens.  We did try adjusting some of the spark / akka timeouts:
> 
>     -Dspark.storage.blockManagerHeartBeatMs=300000
>     -Dspark.akka.frameSize=150
>     -Dspark.akka.timeout=120
>     -Dspark.akka.askTimeout=30
>     -Dspark.akka.logLifecycleEvents=true
> 
> but those settings didn't seem to help the problem at all.  I figure it must be some configuration with the new version of akka that we're missing, but we haven't found anything.  Any ideas?
> 
> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures occur on the tip of the scala-2.10 branch (5429d62d)
> 
> thanks,
> Imran
> 
> 
> 
> -- 
> s
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> 
> -- 
> s
> 
> 
> 
> -- 
> s
> 
> 


Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
pretty sure I found the problem -- two problems actually.  And I think one
of them has been a general lurking problem w/ spark for a while.

1)  we should ignore disassociation events, as you suggested earlier.  They
seem to just indicate a temporary problem, and can generally be ignored.
I've found that they're regularly followed by AssociatedEvents, and it
seems communication really works fine at that point.

2) Task finished messages get lost.  When this message gets sent, we dont'
know it actually gets there:

https://github.com/apache/incubator-spark/blob/scala-2.10/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala#L90

(this is so incredible, I feel I must be overlooking something -- but there
is no ack somewhere else that I'm overlooking, is there??)  So, after the
patch, spark wasn't hanging b/c of the unhandled DisassociatedEvent.  It
hangs b/c the executor has sent some taskFinished messages that never get
received by the driver.  So the driver is waiting for some tasks to finish,
but the executors think they are all done.

I'm gonna add the reliable proxy pattern for this particular interaction
and see if its fixes the problem
http://doc.akka.io/docs/akka/2.2.3/contrib/reliable-proxy.html#introducing-the-reliable-proxy

imran



On Thu, Oct 31, 2013 at 1:17 PM, Imran Rashid <im...@quantifind.com> wrote:

> Hi Prashant,
>
> thanks for looking into this.  I don't have any answers yet, but just
> wanted to send you an update.  I finally figured out how to get all the
> akka logging turned on, so I'm looking at those for more info.  One thing
> immediately jumped out at me -- the Disassociation is actually immediatley
> followed by an Association!  so maybe I came to the wrong conclusion of our
> test of ignoring the DisassociatedEvent.  I'm going to try it again --
> hopefully w/ the logging on, I can find out more about what is going on.  I
> might ask on akka list for help w/ what to look for.  also this thread
> makes me think that it really should just re-associate:
>
> https://groups.google.com/forum/#!searchin/akka-user/Disassociated/akka-user/SajwwbyTriQ/8oxjbZtawxoJ
>
> also, I've noticed that actually these failures are *not* more likely the
> longer the job runs (or not that much, anyway).  when I reorder our jobs, I
> can get the failures pretty regularly within 10 minutes.  It seems to be
> more of a function of what the jobs do.  I'll also see if I can produce a
> simple example which exhibits the same problem.
>
> thanks!
>
> Imran
>
>
>
> On Thu, Oct 31, 2013 at 1:37 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> Hi Imran,
>>
>> So now my jobs are running for more than 18 hours and there is not a
>> single executor failure, I can keep it running for more but I don't think
>> that is going to help. So either you have to give me a way to reproduce the
>> issue (Which would be great !! ) or you have to diagnose for yourself what
>> exactly is causing it, a disconnection usually result in Disassociation
>> event on which we are terminating the executor. In a normal operation this
>> usually means that driver has finished and those log messages are totally
>> fine for that matter. I am not sure why you would not see such a thing in
>> 0.8 release and in scala 2.10, can't say anything until I can see what is
>> happening.
>>
>>
>>
>> On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> Can you show us the sample job ? do you do sc.stop at the end or
>>> System.exit ? Try sc.stop too..
>>>
>>>
>>> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> yeah, just causes them to hang.
>>>>
>>>> the first "deadLetters" message shows up about the same time.  Oddly,
>>>> after it first happens, I keep getting some results trickling in from those
>>>> executors.  (maybe they were just queued up on the driver already, I
>>>> dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
>>>> but the executors are just idle, they're not running anything.
>>>>
>>>> I'm gonna try manually listening for more Association events listed
>>>> here & logging them
>>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
>>>>
>>>> imran
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <scrapcodes@gmail.com
>>>> > wrote:
>>>>
>>>>> I am guessing something wrong with using Dissociation event then.
>>>>>
>>>>> Try applying something on the lines of this patch. This might cause
>>>>> the executors to hang so be prepared for that.
>>>>>
>>>>> diff --git
>>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> index 4e8052a..1ec5d19 100644
>>>>> ---
>>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> +++
>>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>>>>>          executor.launchTask(this, taskDesc.taskId,
>>>>> taskDesc.serializedTask)
>>>>>        }
>>>>>
>>>>> -    case DisassociatedEvent(_, _, _) =>
>>>>> -      logError("Driver terminated or disconnected! Shutting down.")
>>>>> +    case Terminated(actor) =>
>>>>> +      logError("Driver terminated Shutting down.")
>>>>>        System.exit(1)
>>>>> +
>>>>> +    // case DisassociatedEvent(_, _, _) =>
>>>>> +    //   logError("Driver terminated or disconnected! Shutting down.")
>>>>> +    //   System.exit(1)
>>>>>    }
>>>>>
>>>>>    override def statusUpdate(taskId: Long, state: TaskState, data:
>>>>> ByteBuffer) {
>>>>> diff --git
>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> index b6f0ec9..9955484 100644
>>>>> ---
>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> +++
>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
>>>>> ClusterScheduler, actorSystem: Actor
>>>>>        case Terminated(actor) =>
>>>>>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
>>>>> actor terminated"))
>>>>>
>>>>> -      case DisassociatedEvent(_, remoteAddress, _) =>
>>>>> -
>>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>> Akka client disconnected"))
>>>>> +      // case DisassociatedEvent(_, remoteAddress, _) =>
>>>>> +      //
>>>>> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>> Akka client disconnected"))
>>>>>
>>>>>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>>>>>
>>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>> Akka client shutdown"))
>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>> ClusterScheduler, actorSystem: Actor
>>>>>      // Remove a disconnected slave from the cluster
>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>        if (executorActor.contains(executorId)) {
>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>> removing it")
>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>> removing it, reason:" + reason)
>>>>>          val numCores = freeCores(executorId)
>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>> ok, so I applied a few patches
>>>>>>
>>>>>> https://github.com/quantifind/incubator-spark/pull/1/files
>>>>>>
>>>>>> and ran it again, with these options:
>>>>>>
>>>>>> -Dspark.akka.stdout-loglevel=DEBUG \
>>>>>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>>>>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>>>>>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>>>>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>>>>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>>>>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>>>>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>>>>>
>>>>>> On the driver, I see:
>>>>>>
>>>>>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19]
>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>> Actor[akka://spark/deadLetters] to
>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>>>>>> was not delivered. [1] dead letters encountered. This logging can be turned
>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13]
>>>>>> INFO  org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend -
>>>>>> Executor 1 disconnected, so removing it, reason:remote Akka client
>>>>>> disconnected
>>>>>>
>>>>>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13]
>>>>>> ERROR org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1
>>>>>> on dhd2.quantifind.com: remote Akka client disconnected
>>>>>>
>>>>>>
>>>>>> on the worker, stderr:
>>>>>>
>>>>>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>>>>>
>>>>>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>>>>>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>>>>>> spark@ddd0.quantifind.com:36730]
>>>>>>
>>>>>> and unfortunately, all those akka debug options give me *no* useful
>>>>>> info in the worker stdout:
>>>>>>
>>>>>> Starting akka system "sparkExecutor" using config:
>>>>>>
>>>>>>       akka.daemonic = on
>>>>>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>>>>>       akka.stdout-loglevel = "DEBUG"
>>>>>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>>>>>       akka.remote.netty.tcp.transport-class =
>>>>>> "akka.remote.transport.netty.NettyTransport"
>>>>>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>>>>>       akka.remote.netty.tcp.port = 0
>>>>>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>>>>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>>>>>       akka.remote.netty.tcp.execution-pool-size = 4
>>>>>>       akka.actor.default-dispatcher.throughput = 15
>>>>>>       akka.remote.log-remote-lifecycle-events = off
>>>>>>                        akka.remote.log-sent-messages = on
>>>>>> akka.remote.log-received-messages = on
>>>>>> akka.logLevel = DEBUG
>>>>>> akka.actor.debug.autoreceive = on
>>>>>> akka.actor.debug.lifecycle = on
>>>>>> akka.actor.debug.receive = on
>>>>>> akka.log-config-on-start = on
>>>>>> akka.remote.quarantine-systems-for = off
>>>>>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream]
>>>>>> StandardOutLogger started
>>>>>> [DEBUG] [10/30/2013 08:40:30.438]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>>>>>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>>>>>> [DEBUG] [10/30/2013 08:40:30.446]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>>>>>> [DEBUG] [10/30/2013 08:40:30.447]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>> [akka://sparkExecutor/system] started
>>>>>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>>> supervising Actor[akka://sparkExecutor/user]
>>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>>> supervising Actor[akka://sparkExecutor/system]
>>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>>> now monitoring Actor[akka://sparkExecutor/system]
>>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>>>>>> [DEBUG] [10/30/2013 08:40:30.476]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3]
>>>>>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>>>>>> (akka.event.slf4j.Slf4jLogger@24988707)
>>>>>> [DEBUG] [10/30/2013 08:40:30.477]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>> [akka://sparkExecutor/system] now supervising
>>>>>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>>>>>
>>>>>> (followed by similar mesages for the "spark" system)
>>>>>>
>>>>>> I dunno if this means much more to you, but it seems to me that for
>>>>>> some reason the executor decides to disconnect from the master --
>>>>>> unfortunately we don't know why.  I think my logging configuration is not
>>>>>> getting applied correctly, or "log-sent-messages" & "log-received-messages"
>>>>>> don't do what I think they do ... something conflicting must be turing that
>>>>>> logging off.  There are a zillion different remoting settings:
>>>>>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>>>>>
>>>>>> I feel like I really need to get those messages on why it
>>>>>> disconnected to know which ones to play with.  Any ideas for config changes
>>>>>> to see those messages?
>>>>>>
>>>>>> thanks
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <
>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>
>>>>>>> Can you apply this patch too and check the logs of Driver and worker.
>>>>>>>
>>>>>>> diff --git
>>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> index b6f0ec9..ad0ebf7 100644
>>>>>>> ---
>>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> +++
>>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>>>> ClusterScheduler, actorSystem: Actor
>>>>>>>      // Remove a disconnected slave from the cluster
>>>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>>>        if (executorActor.contains(executorId)) {
>>>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>>>> removing it")
>>>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>>>> removing it, reason:" + reason)
>>>>>>>          val numCores = freeCores(executorId)
>>>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>>
>>>>>>>> I just realized something about the failing stages -- they
>>>>>>>> generally occur in steps like this:
>>>>>>>>
>>>>>>>> rdd.mapPartitions{itr =>
>>>>>>>>   val myCounters = initializeSomeDataStructure()
>>>>>>>>   itr.foreach{
>>>>>>>>     //update myCounter in here
>>>>>>>>     ...
>>>>>>>>   }
>>>>>>>>
>>>>>>>>   myCounters.iterator.map{
>>>>>>>>     //some other transformation here ...
>>>>>>>>   }
>>>>>>>> }
>>>>>>>>
>>>>>>>> that is, as a partition is processed, nothing gets output, we just
>>>>>>>> accumulate some values.  Only at the end of the partition do we output some
>>>>>>>> accumulated values.
>>>>>>>>
>>>>>>>> These stages don't always fail, and generally they do succeed after
>>>>>>>> the executor has died and a new one has started -- so I'm pretty confident
>>>>>>>> its not a problem w/ the code.  But maybe we need to add something like a
>>>>>>>> periodic heartbeat in this kind of operation?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <imran@quantifind.com
>>>>>>>> > wrote:
>>>>>>>>
>>>>>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>>>>>> http://akka.io/faq/
>>>>>>>>> and
>>>>>>>>>
>>>>>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>>>>>
>>>>>>>>> unfortunately that will require a patch to spark, but hopefully
>>>>>>>>> that will give us more info to go on ...
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <
>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours
>>>>>>>>>> now without a problem and my jobs write data about the same as you
>>>>>>>>>> suggested. My cluster size is 7 nodes and not *congested* for memory. I
>>>>>>>>>> going to leave jobs running all night long. Meanwhile I had encourage you
>>>>>>>>>> to try to spot the problem such that it is reproducible that can help a ton
>>>>>>>>>> in fixing the issue.
>>>>>>>>>>
>>>>>>>>>> Thanks for testing and reporting your experience. I still feel
>>>>>>>>>> there is something else wrong !. About tolerance for network connection
>>>>>>>>>> timeouts, setting those properties should work, but I am afraid about
>>>>>>>>>> Disassociation Event though. I will have to check this is indeed hard to
>>>>>>>>>> reproduce bug if it is, I mean how do I simulate network delays ?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <
>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>>>>>>
>>>>>>>>>>> At first I thought it must be some temporary network problem too
>>>>>>>>>>> -- but the times between receiving task completion events from an executor
>>>>>>>>>>> and declaring it failed are really small, so I didn't think that could
>>>>>>>>>>> possibly be it.  Plus we tried increasing various akka timeouts, but that
>>>>>>>>>>> didn't help.  Or maybe there are some other spark / akka properities we
>>>>>>>>>>> should be setting?  It certainly should be resilient to such a temporary
>>>>>>>>>>> network issue, if that is the problem.
>>>>>>>>>>>
>>>>>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>>>>>> total for the entire stage).
>>>>>>>>>>>
>>>>>>>>>>> thanks
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Are you using mesos ? I admit to have not properly tested
>>>>>>>>>>>> things on mesos though.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually
>>>>>>>>>>>>> seen when a node is disassociated with other by either a network failure or
>>>>>>>>>>>>> even clean shutdown. This suggests some network issue to me, are you
>>>>>>>>>>>>> running on EC2 ? It might be a temporary thing in that case.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I had like to have more details on the long jobs though, how
>>>>>>>>>>>>> long ?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 22013-10-29 11:03:54,577
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>> Message
>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:54,579
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>> Message [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>>>>>> new one is started:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,775
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-3] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>> Message
>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not
>>>>>>>>>>>>>> delivered. [10] dead letters encountered, no more dead letters will be
>>>>>>>>>>>>>> logged. This logging can be turned off or adjusted with configuration
>>>>>>>>>>>>>> settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,778
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter -
>>>>>>>>>>>>>> AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068]
>>>>>>>>>>>>>> -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]:
>>>>>>>>>>>>>> Error [Association failed with [akka.tcp://
>>>>>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed
>>>>>>>>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>>>>>> Caused by:
>>>>>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [1] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [2] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [3] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [4] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>>>>>> while the others sit around.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I have no idea what is triggering this behavior -- there
>>>>>>>>>>>>>> isn't any particular point in the job that it regularly occurs at.  Certain
>>>>>>>>>>>>>> steps seem more prone to this, but there isn't any step which regularly
>>>>>>>>>>>>>> causes the problem.  In a long pipeline of steps, though, that loop becomes
>>>>>>>>>>>>>> very likely.  I don't think its a timeout issue -- the initial failing
>>>>>>>>>>>>>> executors can be actively completing stages just seconds before this
>>>>>>>>>>>>>> failure happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> thanks,
>>>>>>>>>>>>>> Imran
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --
>>>>>>>>>>>>> s
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> --
>>>>>>>>>>>> s
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> s
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> s
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> s
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>>
>> --
>> s
>>
>
>

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
And as Matei said, did you try increasing this
akka.remote.watch-failure-detector.acceptable-heartbeat-pause
?


On Fri, Nov 1, 2013 at 9:24 AM, Prashant Sharma <sc...@gmail.com>wrote:

> Hi Imran,
>
> Thanks for your patience and testing, see inline.
>
> On Thu, Oct 31, 2013 at 11:47 PM, Imran Rashid <im...@quantifind.com>wrote:
>
>> Hi Prashant,
>>
>> thanks for looking into this.  I don't have any answers yet, but just
>> wanted to send you an update.  I finally figured out how to get all the
>> akka logging turned on, so I'm looking at those for more info.  One thing
>> immediately jumped out at me -- the Disassociation is actually immediatley
>> followed by an Association!  so maybe I came to the wrong conclusion of our
>> test of ignoring the DisassociatedEvent.
>>
> Before we conclude something about reliable messaging, I want you to for
> once consider other possibilities like actual network reconnection and may
> be a GC pause ? Try connecting something like jconsole (or alike ) and see
> what happens on the driver and executor.
>
> My doubt are since we are using standalone mode where even master and
> worker are also actors then if we see a weird behaviour on the executor and
> driver then Why not on master and worker too ? They should also break away
> from each other. For this reason, I am doubting our conclusions and may be
> if we narrow down the problem first before we conclude something. It is a
> regression in akka 2.2.3 it uses more memory than it used to be in 2.1.x.
> See https://github.com/akka/akka/issues/1810
>
>
> I'm going to try it again -- hopefully w/ the logging on, I can find out
>> more about what is going on.  I might ask on akka list for help w/ what to
>> look for.  also this thread makes me think that it really should just
>> re-associate:
>>
>> https://groups.google.com/forum/#!searchin/akka-user/Disassociated/akka-user/SajwwbyTriQ/8oxjbZtawxoJ
>>
>> also, I've noticed that actually these failures are *not* more likely the
>> longer the job runs (or not that much, anyway).  when I reorder our jobs, I
>> can get the failures pretty regularly within 10 minutes.  It seems to be
>> more of a function of what the jobs do.  I'll also see if I can produce a
>> simple example which exhibits the same problem.
>>
>> Like pointed on that thread one possibility is that we are hogging too
> many threads but I guess, the systems you are using are really powerful but
> then try leaving 1 or more CPU free on each worker nodes. (Ofcourse this
> not my suggested solution, but a way to narrow down that hogging threads
> might be an issue.)
>
> I have got some hints now will think more.
>
> thanks!
>>
>> Imran
>>
>>
>>
>> On Thu, Oct 31, 2013 at 1:37 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> Hi Imran,
>>>
>>> So now my jobs are running for more than 18 hours and there is not a
>>> single executor failure, I can keep it running for more but I don't think
>>> that is going to help. So either you have to give me a way to reproduce the
>>> issue (Which would be great !! ) or you have to diagnose for yourself what
>>> exactly is causing it, a disconnection usually result in Disassociation
>>> event on which we are terminating the executor. In a normal operation this
>>> usually means that driver has finished and those log messages are totally
>>> fine for that matter. I am not sure why you would not see such a thing in
>>> 0.8 release and in scala 2.10, can't say anything until I can see what is
>>> happening.
>>>
>>>
>>>
>>> On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>
>>>> Can you show us the sample job ? do you do sc.stop at the end or
>>>> System.exit ? Try sc.stop too..
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>> yeah, just causes them to hang.
>>>>>
>>>>> the first "deadLetters" message shows up about the same time.  Oddly,
>>>>> after it first happens, I keep getting some results trickling in from those
>>>>> executors.  (maybe they were just queued up on the driver already, I
>>>>> dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
>>>>> but the executors are just idle, they're not running anything.
>>>>>
>>>>> I'm gonna try manually listening for more Association events listed
>>>>> here & logging them
>>>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
>>>>>
>>>>> imran
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <
>>>>> scrapcodes@gmail.com> wrote:
>>>>>
>>>>>> I am guessing something wrong with using Dissociation event then.
>>>>>>
>>>>>> Try applying something on the lines of this patch. This might cause
>>>>>> the executors to hang so be prepared for that.
>>>>>>
>>>>>> diff --git
>>>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>>> index 4e8052a..1ec5d19 100644
>>>>>> ---
>>>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>>> +++
>>>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>>> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>>>>>>          executor.launchTask(this, taskDesc.taskId,
>>>>>> taskDesc.serializedTask)
>>>>>>        }
>>>>>>
>>>>>> -    case DisassociatedEvent(_, _, _) =>
>>>>>> -      logError("Driver terminated or disconnected! Shutting down.")
>>>>>> +    case Terminated(actor) =>
>>>>>> +      logError("Driver terminated Shutting down.")
>>>>>>        System.exit(1)
>>>>>> +
>>>>>> +    // case DisassociatedEvent(_, _, _) =>
>>>>>> +    //   logError("Driver terminated or disconnected! Shutting
>>>>>> down.")
>>>>>> +    //   System.exit(1)
>>>>>>    }
>>>>>>
>>>>>>    override def statusUpdate(taskId: Long, state: TaskState, data:
>>>>>> ByteBuffer) {
>>>>>> diff --git
>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> index b6f0ec9..9955484 100644
>>>>>> ---
>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> +++
>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
>>>>>> ClusterScheduler, actorSystem: Actor
>>>>>>        case Terminated(actor) =>
>>>>>>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
>>>>>> actor terminated"))
>>>>>>
>>>>>> -      case DisassociatedEvent(_, remoteAddress, _) =>
>>>>>> -
>>>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>>> Akka client disconnected"))
>>>>>> +      // case DisassociatedEvent(_, remoteAddress, _) =>
>>>>>> +      //
>>>>>> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>>> Akka client disconnected"))
>>>>>>
>>>>>>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>>>>>>
>>>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>>> Akka client shutdown"))
>>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>>> ClusterScheduler, actorSystem: Actor
>>>>>>      // Remove a disconnected slave from the cluster
>>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>>        if (executorActor.contains(executorId)) {
>>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>>> removing it")
>>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>>> removing it, reason:" + reason)
>>>>>>          val numCores = freeCores(executorId)
>>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>
>>>>>>> ok, so I applied a few patches
>>>>>>>
>>>>>>> https://github.com/quantifind/incubator-spark/pull/1/files
>>>>>>>
>>>>>>> and ran it again, with these options:
>>>>>>>
>>>>>>> -Dspark.akka.stdout-loglevel=DEBUG \
>>>>>>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>>>>>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>>>>>>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>>>>>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>>>>>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>>>>>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>>>>>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>>>>>>
>>>>>>> On the driver, I see:
>>>>>>>
>>>>>>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19]
>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>>>>>>> was not delivered. [1] dead letters encountered. This logging can be turned
>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13]
>>>>>>> INFO  org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend -
>>>>>>> Executor 1 disconnected, so removing it, reason:remote Akka client
>>>>>>> disconnected
>>>>>>>
>>>>>>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13]
>>>>>>> ERROR org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1
>>>>>>> on dhd2.quantifind.com: remote Akka client disconnected
>>>>>>>
>>>>>>>
>>>>>>> on the worker, stderr:
>>>>>>>
>>>>>>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>>>>>>
>>>>>>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>>>>>>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>>>>>>> spark@ddd0.quantifind.com:36730]
>>>>>>>
>>>>>>> and unfortunately, all those akka debug options give me *no* useful
>>>>>>> info in the worker stdout:
>>>>>>>
>>>>>>> Starting akka system "sparkExecutor" using config:
>>>>>>>
>>>>>>>       akka.daemonic = on
>>>>>>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>>>>>>       akka.stdout-loglevel = "DEBUG"
>>>>>>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>>>>>>       akka.remote.netty.tcp.transport-class =
>>>>>>> "akka.remote.transport.netty.NettyTransport"
>>>>>>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>>>>>>       akka.remote.netty.tcp.port = 0
>>>>>>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>>>>>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>>>>>>       akka.remote.netty.tcp.execution-pool-size = 4
>>>>>>>       akka.actor.default-dispatcher.throughput = 15
>>>>>>>       akka.remote.log-remote-lifecycle-events = off
>>>>>>>                        akka.remote.log-sent-messages = on
>>>>>>> akka.remote.log-received-messages = on
>>>>>>> akka.logLevel = DEBUG
>>>>>>> akka.actor.debug.autoreceive = on
>>>>>>> akka.actor.debug.lifecycle = on
>>>>>>> akka.actor.debug.receive = on
>>>>>>> akka.log-config-on-start = on
>>>>>>> akka.remote.quarantine-systems-for = off
>>>>>>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream]
>>>>>>> StandardOutLogger started
>>>>>>> [DEBUG] [10/30/2013 08:40:30.438]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>>>>>>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>>>>>>> [DEBUG] [10/30/2013 08:40:30.446]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>>>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>>>>>>> [DEBUG] [10/30/2013 08:40:30.447]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>>> [akka://sparkExecutor/system] started
>>>>>>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>>>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>>>> supervising Actor[akka://sparkExecutor/user]
>>>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>>>> supervising Actor[akka://sparkExecutor/system]
>>>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>>>> now monitoring Actor[akka://sparkExecutor/system]
>>>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>>>>>>> [DEBUG] [10/30/2013 08:40:30.476]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3]
>>>>>>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>>>>>>> (akka.event.slf4j.Slf4jLogger@24988707)
>>>>>>> [DEBUG] [10/30/2013 08:40:30.477]
>>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>>> [akka://sparkExecutor/system] now supervising
>>>>>>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>>>>>>
>>>>>>> (followed by similar mesages for the "spark" system)
>>>>>>>
>>>>>>> I dunno if this means much more to you, but it seems to me that for
>>>>>>> some reason the executor decides to disconnect from the master --
>>>>>>> unfortunately we don't know why.  I think my logging configuration is not
>>>>>>> getting applied correctly, or "log-sent-messages" & "log-received-messages"
>>>>>>> don't do what I think they do ... something conflicting must be turing that
>>>>>>> logging off.  There are a zillion different remoting settings:
>>>>>>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>>>>>>
>>>>>>> I feel like I really need to get those messages on why it
>>>>>>> disconnected to know which ones to play with.  Any ideas for config changes
>>>>>>> to see those messages?
>>>>>>>
>>>>>>> thanks
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <
>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>
>>>>>>>> Can you apply this patch too and check the logs of Driver and
>>>>>>>> worker.
>>>>>>>>
>>>>>>>> diff --git
>>>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>>> index b6f0ec9..ad0ebf7 100644
>>>>>>>> ---
>>>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>>> +++
>>>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>>>>> ClusterScheduler, actorSystem: Actor
>>>>>>>>      // Remove a disconnected slave from the cluster
>>>>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>>>>        if (executorActor.contains(executorId)) {
>>>>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>>>>> removing it")
>>>>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>>>>> removing it, reason:" + reason)
>>>>>>>>          val numCores = freeCores(executorId)
>>>>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <imran@quantifind.com
>>>>>>>> > wrote:
>>>>>>>>
>>>>>>>>> I just realized something about the failing stages -- they
>>>>>>>>> generally occur in steps like this:
>>>>>>>>>
>>>>>>>>> rdd.mapPartitions{itr =>
>>>>>>>>>   val myCounters = initializeSomeDataStructure()
>>>>>>>>>   itr.foreach{
>>>>>>>>>     //update myCounter in here
>>>>>>>>>     ...
>>>>>>>>>   }
>>>>>>>>>
>>>>>>>>>   myCounters.iterator.map{
>>>>>>>>>     //some other transformation here ...
>>>>>>>>>   }
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> that is, as a partition is processed, nothing gets output, we just
>>>>>>>>> accumulate some values.  Only at the end of the partition do we output some
>>>>>>>>> accumulated values.
>>>>>>>>>
>>>>>>>>> These stages don't always fail, and generally they do succeed
>>>>>>>>> after the executor has died and a new one has started -- so I'm pretty
>>>>>>>>> confident its not a problem w/ the code.  But maybe we need to add
>>>>>>>>> something like a periodic heartbeat in this kind of operation?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <
>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>
>>>>>>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>>>>>>> http://akka.io/faq/
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>>>>>>
>>>>>>>>>> unfortunately that will require a patch to spark, but hopefully
>>>>>>>>>> that will give us more info to go on ...
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <
>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> I have things running (from scala 2.10 branch) for over 3-4
>>>>>>>>>>> hours now without a problem and my jobs write data about the same as you
>>>>>>>>>>> suggested. My cluster size is 7 nodes and not *congested* for memory. I
>>>>>>>>>>> going to leave jobs running all night long. Meanwhile I had encourage you
>>>>>>>>>>> to try to spot the problem such that it is reproducible that can help a ton
>>>>>>>>>>> in fixing the issue.
>>>>>>>>>>>
>>>>>>>>>>> Thanks for testing and reporting your experience. I still feel
>>>>>>>>>>> there is something else wrong !. About tolerance for network connection
>>>>>>>>>>> timeouts, setting those properties should work, but I am afraid about
>>>>>>>>>>> Disassociation Event though. I will have to check this is indeed hard to
>>>>>>>>>>> reproduce bug if it is, I mean how do I simulate network delays ?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <
>>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> This is a spark-standalone setup (not mesos), on our own
>>>>>>>>>>>> cluster.
>>>>>>>>>>>>
>>>>>>>>>>>> At first I thought it must be some temporary network problem
>>>>>>>>>>>> too -- but the times between receiving task completion events from an
>>>>>>>>>>>> executor and declaring it failed are really small, so I didn't think that
>>>>>>>>>>>> could possibly be it.  Plus we tried increasing various akka timeouts, but
>>>>>>>>>>>> that didn't help.  Or maybe there are some other spark / akka properities
>>>>>>>>>>>> we should be setting?  It certainly should be resilient to such a temporary
>>>>>>>>>>>> network issue, if that is the problem.
>>>>>>>>>>>>
>>>>>>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>>>>>>> total for the entire stage).
>>>>>>>>>>>>
>>>>>>>>>>>> thanks
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Are you using mesos ? I admit to have not properly tested
>>>>>>>>>>>>> things on mesos though.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually
>>>>>>>>>>>>>> seen when a node is disassociated with other by either a network failure or
>>>>>>>>>>>>>> even clean shutdown. This suggests some network issue to me, are you
>>>>>>>>>>>>>> running on EC2 ? It might be a temporary thing in that case.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I had like to have more details on the long jobs though, how
>>>>>>>>>>>>>> long ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The first suspicious thing is that we get akka warnings
>>>>>>>>>>>>>>> about undeliverable messages sent to deadLetters:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 22013-10-29 11:03:54,577
>>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>>> Message
>>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2013-10-29 11:03:54,579
>>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>>> Message [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>>>>>>> new one is started:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2013-10-29 11:03:58,775
>>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-3] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>>> Message
>>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not
>>>>>>>>>>>>>>> delivered. [10] dead letters encountered, no more dead letters will be
>>>>>>>>>>>>>>> logged. This logging can be turned off or adjusted with configuration
>>>>>>>>>>>>>>> settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2013-10-29 11:03:58,778
>>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter -
>>>>>>>>>>>>>>> AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068]
>>>>>>>>>>>>>>> -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]:
>>>>>>>>>>>>>>> Error [Association failed with [akka.tcp://
>>>>>>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed
>>>>>>>>>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>>>>>>> Caused by:
>>>>>>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID
>>>>>>>>>>>>>>> 943
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>>> delivered. [1] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>>> delivered. [2] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>>> delivered. [3] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>>> delivered. [4] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>>>>>>> while the others sit around.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I have no idea what is triggering this behavior -- there
>>>>>>>>>>>>>>> isn't any particular point in the job that it regularly occurs at.  Certain
>>>>>>>>>>>>>>> steps seem more prone to this, but there isn't any step which regularly
>>>>>>>>>>>>>>> causes the problem.  In a long pipeline of steps, though, that loop becomes
>>>>>>>>>>>>>>> very likely.  I don't think its a timeout issue -- the initial failing
>>>>>>>>>>>>>>> executors can be actively completing stages just seconds before this
>>>>>>>>>>>>>>> failure happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> but those settings didn't seem to help the problem at all.
>>>>>>>>>>>>>>> I figure it must be some configuration with the new version of akka that
>>>>>>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.
>>>>>>>>>>>>>>> The failures occur on the tip of the scala-2.10 branch (
>>>>>>>>>>>>>>> 5429d62d)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> thanks,
>>>>>>>>>>>>>>> Imran
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --
>>>>>>>>>>>>>> s
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --
>>>>>>>>>>>>> s
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> s
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> s
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> s
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> s
>>>>
>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>
>
> --
> s
>



-- 
s

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Hi Imran,

Thanks for your patience and testing, see inline.

On Thu, Oct 31, 2013 at 11:47 PM, Imran Rashid <im...@quantifind.com> wrote:

> Hi Prashant,
>
> thanks for looking into this.  I don't have any answers yet, but just
> wanted to send you an update.  I finally figured out how to get all the
> akka logging turned on, so I'm looking at those for more info.  One thing
> immediately jumped out at me -- the Disassociation is actually immediatley
> followed by an Association!  so maybe I came to the wrong conclusion of our
> test of ignoring the DisassociatedEvent.
>
Before we conclude something about reliable messaging, I want you to for
once consider other possibilities like actual network reconnection and may
be a GC pause ? Try connecting something like jconsole (or alike ) and see
what happens on the driver and executor.

My doubt are since we are using standalone mode where even master and
worker are also actors then if we see a weird behaviour on the executor and
driver then Why not on master and worker too ? They should also break away
from each other. For this reason, I am doubting our conclusions and may be
if we narrow down the problem first before we conclude something. It is a
regression in akka 2.2.3 it uses more memory than it used to be in 2.1.x.
See https://github.com/akka/akka/issues/1810


I'm going to try it again -- hopefully w/ the logging on, I can find out
> more about what is going on.  I might ask on akka list for help w/ what to
> look for.  also this thread makes me think that it really should just
> re-associate:
>
> https://groups.google.com/forum/#!searchin/akka-user/Disassociated/akka-user/SajwwbyTriQ/8oxjbZtawxoJ
>
> also, I've noticed that actually these failures are *not* more likely the
> longer the job runs (or not that much, anyway).  when I reorder our jobs, I
> can get the failures pretty regularly within 10 minutes.  It seems to be
> more of a function of what the jobs do.  I'll also see if I can produce a
> simple example which exhibits the same problem.
>
> Like pointed on that thread one possibility is that we are hogging too
many threads but I guess, the systems you are using are really powerful but
then try leaving 1 or more CPU free on each worker nodes. (Ofcourse this
not my suggested solution, but a way to narrow down that hogging threads
might be an issue.)

I have got some hints now will think more.

thanks!
>
> Imran
>
>
>
> On Thu, Oct 31, 2013 at 1:37 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> Hi Imran,
>>
>> So now my jobs are running for more than 18 hours and there is not a
>> single executor failure, I can keep it running for more but I don't think
>> that is going to help. So either you have to give me a way to reproduce the
>> issue (Which would be great !! ) or you have to diagnose for yourself what
>> exactly is causing it, a disconnection usually result in Disassociation
>> event on which we are terminating the executor. In a normal operation this
>> usually means that driver has finished and those log messages are totally
>> fine for that matter. I am not sure why you would not see such a thing in
>> 0.8 release and in scala 2.10, can't say anything until I can see what is
>> happening.
>>
>>
>>
>> On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> Can you show us the sample job ? do you do sc.stop at the end or
>>> System.exit ? Try sc.stop too..
>>>
>>>
>>> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> yeah, just causes them to hang.
>>>>
>>>> the first "deadLetters" message shows up about the same time.  Oddly,
>>>> after it first happens, I keep getting some results trickling in from those
>>>> executors.  (maybe they were just queued up on the driver already, I
>>>> dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
>>>> but the executors are just idle, they're not running anything.
>>>>
>>>> I'm gonna try manually listening for more Association events listed
>>>> here & logging them
>>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
>>>>
>>>> imran
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <scrapcodes@gmail.com
>>>> > wrote:
>>>>
>>>>> I am guessing something wrong with using Dissociation event then.
>>>>>
>>>>> Try applying something on the lines of this patch. This might cause
>>>>> the executors to hang so be prepared for that.
>>>>>
>>>>> diff --git
>>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> index 4e8052a..1ec5d19 100644
>>>>> ---
>>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> +++
>>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>>> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>>>>>          executor.launchTask(this, taskDesc.taskId,
>>>>> taskDesc.serializedTask)
>>>>>        }
>>>>>
>>>>> -    case DisassociatedEvent(_, _, _) =>
>>>>> -      logError("Driver terminated or disconnected! Shutting down.")
>>>>> +    case Terminated(actor) =>
>>>>> +      logError("Driver terminated Shutting down.")
>>>>>        System.exit(1)
>>>>> +
>>>>> +    // case DisassociatedEvent(_, _, _) =>
>>>>> +    //   logError("Driver terminated or disconnected! Shutting down.")
>>>>> +    //   System.exit(1)
>>>>>    }
>>>>>
>>>>>    override def statusUpdate(taskId: Long, state: TaskState, data:
>>>>> ByteBuffer) {
>>>>> diff --git
>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> index b6f0ec9..9955484 100644
>>>>> ---
>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> +++
>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
>>>>> ClusterScheduler, actorSystem: Actor
>>>>>        case Terminated(actor) =>
>>>>>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
>>>>> actor terminated"))
>>>>>
>>>>> -      case DisassociatedEvent(_, remoteAddress, _) =>
>>>>> -
>>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>> Akka client disconnected"))
>>>>> +      // case DisassociatedEvent(_, remoteAddress, _) =>
>>>>> +      //
>>>>> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>> Akka client disconnected"))
>>>>>
>>>>>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>>>>>
>>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>>> Akka client shutdown"))
>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>> ClusterScheduler, actorSystem: Actor
>>>>>      // Remove a disconnected slave from the cluster
>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>        if (executorActor.contains(executorId)) {
>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>> removing it")
>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>> removing it, reason:" + reason)
>>>>>          val numCores = freeCores(executorId)
>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>> ok, so I applied a few patches
>>>>>>
>>>>>> https://github.com/quantifind/incubator-spark/pull/1/files
>>>>>>
>>>>>> and ran it again, with these options:
>>>>>>
>>>>>> -Dspark.akka.stdout-loglevel=DEBUG \
>>>>>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>>>>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>>>>>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>>>>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>>>>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>>>>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>>>>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>>>>>
>>>>>> On the driver, I see:
>>>>>>
>>>>>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19]
>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>> Actor[akka://spark/deadLetters] to
>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>>>>>> was not delivered. [1] dead letters encountered. This logging can be turned
>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13]
>>>>>> INFO  org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend -
>>>>>> Executor 1 disconnected, so removing it, reason:remote Akka client
>>>>>> disconnected
>>>>>>
>>>>>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13]
>>>>>> ERROR org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1
>>>>>> on dhd2.quantifind.com: remote Akka client disconnected
>>>>>>
>>>>>>
>>>>>> on the worker, stderr:
>>>>>>
>>>>>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>>>>>
>>>>>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>>>>>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>>>>>> spark@ddd0.quantifind.com:36730]
>>>>>>
>>>>>> and unfortunately, all those akka debug options give me *no* useful
>>>>>> info in the worker stdout:
>>>>>>
>>>>>> Starting akka system "sparkExecutor" using config:
>>>>>>
>>>>>>       akka.daemonic = on
>>>>>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>>>>>       akka.stdout-loglevel = "DEBUG"
>>>>>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>>>>>       akka.remote.netty.tcp.transport-class =
>>>>>> "akka.remote.transport.netty.NettyTransport"
>>>>>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>>>>>       akka.remote.netty.tcp.port = 0
>>>>>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>>>>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>>>>>       akka.remote.netty.tcp.execution-pool-size = 4
>>>>>>       akka.actor.default-dispatcher.throughput = 15
>>>>>>       akka.remote.log-remote-lifecycle-events = off
>>>>>>                        akka.remote.log-sent-messages = on
>>>>>> akka.remote.log-received-messages = on
>>>>>> akka.logLevel = DEBUG
>>>>>> akka.actor.debug.autoreceive = on
>>>>>> akka.actor.debug.lifecycle = on
>>>>>> akka.actor.debug.receive = on
>>>>>> akka.log-config-on-start = on
>>>>>> akka.remote.quarantine-systems-for = off
>>>>>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream]
>>>>>> StandardOutLogger started
>>>>>> [DEBUG] [10/30/2013 08:40:30.438]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>>>>>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>>>>>> [DEBUG] [10/30/2013 08:40:30.446]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>>>>>> [DEBUG] [10/30/2013 08:40:30.447]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>> [akka://sparkExecutor/system] started
>>>>>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>>> supervising Actor[akka://sparkExecutor/user]
>>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>>> supervising Actor[akka://sparkExecutor/system]
>>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>>> now monitoring Actor[akka://sparkExecutor/system]
>>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>>>>>> [DEBUG] [10/30/2013 08:40:30.476]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-3]
>>>>>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>>>>>> (akka.event.slf4j.Slf4jLogger@24988707)
>>>>>> [DEBUG] [10/30/2013 08:40:30.477]
>>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>>> [akka://sparkExecutor/system] now supervising
>>>>>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>>>>>
>>>>>> (followed by similar mesages for the "spark" system)
>>>>>>
>>>>>> I dunno if this means much more to you, but it seems to me that for
>>>>>> some reason the executor decides to disconnect from the master --
>>>>>> unfortunately we don't know why.  I think my logging configuration is not
>>>>>> getting applied correctly, or "log-sent-messages" & "log-received-messages"
>>>>>> don't do what I think they do ... something conflicting must be turing that
>>>>>> logging off.  There are a zillion different remoting settings:
>>>>>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>>>>>
>>>>>> I feel like I really need to get those messages on why it
>>>>>> disconnected to know which ones to play with.  Any ideas for config changes
>>>>>> to see those messages?
>>>>>>
>>>>>> thanks
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <
>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>
>>>>>>> Can you apply this patch too and check the logs of Driver and worker.
>>>>>>>
>>>>>>> diff --git
>>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> index b6f0ec9..ad0ebf7 100644
>>>>>>> ---
>>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> +++
>>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>>>> ClusterScheduler, actorSystem: Actor
>>>>>>>      // Remove a disconnected slave from the cluster
>>>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>>>        if (executorActor.contains(executorId)) {
>>>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>>>> removing it")
>>>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>>>> removing it, reason:" + reason)
>>>>>>>          val numCores = freeCores(executorId)
>>>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>>
>>>>>>>> I just realized something about the failing stages -- they
>>>>>>>> generally occur in steps like this:
>>>>>>>>
>>>>>>>> rdd.mapPartitions{itr =>
>>>>>>>>   val myCounters = initializeSomeDataStructure()
>>>>>>>>   itr.foreach{
>>>>>>>>     //update myCounter in here
>>>>>>>>     ...
>>>>>>>>   }
>>>>>>>>
>>>>>>>>   myCounters.iterator.map{
>>>>>>>>     //some other transformation here ...
>>>>>>>>   }
>>>>>>>> }
>>>>>>>>
>>>>>>>> that is, as a partition is processed, nothing gets output, we just
>>>>>>>> accumulate some values.  Only at the end of the partition do we output some
>>>>>>>> accumulated values.
>>>>>>>>
>>>>>>>> These stages don't always fail, and generally they do succeed after
>>>>>>>> the executor has died and a new one has started -- so I'm pretty confident
>>>>>>>> its not a problem w/ the code.  But maybe we need to add something like a
>>>>>>>> periodic heartbeat in this kind of operation?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <imran@quantifind.com
>>>>>>>> > wrote:
>>>>>>>>
>>>>>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>>>>>> http://akka.io/faq/
>>>>>>>>> and
>>>>>>>>>
>>>>>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>>>>>
>>>>>>>>> unfortunately that will require a patch to spark, but hopefully
>>>>>>>>> that will give us more info to go on ...
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <
>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours
>>>>>>>>>> now without a problem and my jobs write data about the same as you
>>>>>>>>>> suggested. My cluster size is 7 nodes and not *congested* for memory. I
>>>>>>>>>> going to leave jobs running all night long. Meanwhile I had encourage you
>>>>>>>>>> to try to spot the problem such that it is reproducible that can help a ton
>>>>>>>>>> in fixing the issue.
>>>>>>>>>>
>>>>>>>>>> Thanks for testing and reporting your experience. I still feel
>>>>>>>>>> there is something else wrong !. About tolerance for network connection
>>>>>>>>>> timeouts, setting those properties should work, but I am afraid about
>>>>>>>>>> Disassociation Event though. I will have to check this is indeed hard to
>>>>>>>>>> reproduce bug if it is, I mean how do I simulate network delays ?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <
>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>>>>>>
>>>>>>>>>>> At first I thought it must be some temporary network problem too
>>>>>>>>>>> -- but the times between receiving task completion events from an executor
>>>>>>>>>>> and declaring it failed are really small, so I didn't think that could
>>>>>>>>>>> possibly be it.  Plus we tried increasing various akka timeouts, but that
>>>>>>>>>>> didn't help.  Or maybe there are some other spark / akka properities we
>>>>>>>>>>> should be setting?  It certainly should be resilient to such a temporary
>>>>>>>>>>> network issue, if that is the problem.
>>>>>>>>>>>
>>>>>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>>>>>> total for the entire stage).
>>>>>>>>>>>
>>>>>>>>>>> thanks
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Are you using mesos ? I admit to have not properly tested
>>>>>>>>>>>> things on mesos though.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually
>>>>>>>>>>>>> seen when a node is disassociated with other by either a network failure or
>>>>>>>>>>>>> even clean shutdown. This suggests some network issue to me, are you
>>>>>>>>>>>>> running on EC2 ? It might be a temporary thing in that case.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I had like to have more details on the long jobs though, how
>>>>>>>>>>>>> long ?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 22013-10-29 11:03:54,577
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>> Message
>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:54,579
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>> Message [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>>>>>> new one is started:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,775
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-3] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>>> Message
>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not
>>>>>>>>>>>>>> delivered. [10] dead letters encountered, no more dead letters will be
>>>>>>>>>>>>>> logged. This logging can be turned off or adjusted with configuration
>>>>>>>>>>>>>> settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,778
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter -
>>>>>>>>>>>>>> AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068]
>>>>>>>>>>>>>> -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]:
>>>>>>>>>>>>>> Error [Association failed with [akka.tcp://
>>>>>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed
>>>>>>>>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>>>>>> Caused by:
>>>>>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [1] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [2] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [3] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not
>>>>>>>>>>>>>> delivered. [4] dead letters encountered. This logging can be turned off or
>>>>>>>>>>>>>> adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>>>>>> while the others sit around.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I have no idea what is triggering this behavior -- there
>>>>>>>>>>>>>> isn't any particular point in the job that it regularly occurs at.  Certain
>>>>>>>>>>>>>> steps seem more prone to this, but there isn't any step which regularly
>>>>>>>>>>>>>> causes the problem.  In a long pipeline of steps, though, that loop becomes
>>>>>>>>>>>>>> very likely.  I don't think its a timeout issue -- the initial failing
>>>>>>>>>>>>>> executors can be actively completing stages just seconds before this
>>>>>>>>>>>>>> failure happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> thanks,
>>>>>>>>>>>>>> Imran
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --
>>>>>>>>>>>>> s
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> --
>>>>>>>>>>>> s
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> s
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> s
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> s
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>>
>> --
>> s
>>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
Hi Prashant,

thanks for looking into this.  I don't have any answers yet, but just
wanted to send you an update.  I finally figured out how to get all the
akka logging turned on, so I'm looking at those for more info.  One thing
immediately jumped out at me -- the Disassociation is actually immediatley
followed by an Association!  so maybe I came to the wrong conclusion of our
test of ignoring the DisassociatedEvent.  I'm going to try it again --
hopefully w/ the logging on, I can find out more about what is going on.  I
might ask on akka list for help w/ what to look for.  also this thread
makes me think that it really should just re-associate:
https://groups.google.com/forum/#!searchin/akka-user/Disassociated/akka-user/SajwwbyTriQ/8oxjbZtawxoJ

also, I've noticed that actually these failures are *not* more likely the
longer the job runs (or not that much, anyway).  when I reorder our jobs, I
can get the failures pretty regularly within 10 minutes.  It seems to be
more of a function of what the jobs do.  I'll also see if I can produce a
simple example which exhibits the same problem.

thanks!

Imran



On Thu, Oct 31, 2013 at 1:37 AM, Prashant Sharma <sc...@gmail.com>wrote:

> Hi Imran,
>
> So now my jobs are running for more than 18 hours and there is not a
> single executor failure, I can keep it running for more but I don't think
> that is going to help. So either you have to give me a way to reproduce the
> issue (Which would be great !! ) or you have to diagnose for yourself what
> exactly is causing it, a disconnection usually result in Disassociation
> event on which we are terminating the executor. In a normal operation this
> usually means that driver has finished and those log messages are totally
> fine for that matter. I am not sure why you would not see such a thing in
> 0.8 release and in scala 2.10, can't say anything until I can see what is
> happening.
>
>
>
> On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> Can you show us the sample job ? do you do sc.stop at the end or
>> System.exit ? Try sc.stop too..
>>
>>
>> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> yeah, just causes them to hang.
>>>
>>> the first "deadLetters" message shows up about the same time.  Oddly,
>>> after it first happens, I keep getting some results trickling in from those
>>> executors.  (maybe they were just queued up on the driver already, I
>>> dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
>>> but the executors are just idle, they're not running anything.
>>>
>>> I'm gonna try manually listening for more Association events listed here
>>> & logging them
>>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
>>>
>>> imran
>>>
>>>
>>>
>>>
>>> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>
>>>> I am guessing something wrong with using Dissociation event then.
>>>>
>>>> Try applying something on the lines of this patch. This might cause the
>>>> executors to hang so be prepared for that.
>>>>
>>>> diff --git
>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>> index 4e8052a..1ec5d19 100644
>>>> ---
>>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>> +++
>>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>>> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>>>>          executor.launchTask(this, taskDesc.taskId,
>>>> taskDesc.serializedTask)
>>>>        }
>>>>
>>>> -    case DisassociatedEvent(_, _, _) =>
>>>> -      logError("Driver terminated or disconnected! Shutting down.")
>>>> +    case Terminated(actor) =>
>>>> +      logError("Driver terminated Shutting down.")
>>>>        System.exit(1)
>>>> +
>>>> +    // case DisassociatedEvent(_, _, _) =>
>>>> +    //   logError("Driver terminated or disconnected! Shutting down.")
>>>> +    //   System.exit(1)
>>>>    }
>>>>
>>>>    override def statusUpdate(taskId: Long, state: TaskState, data:
>>>> ByteBuffer) {
>>>> diff --git
>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> index b6f0ec9..9955484 100644
>>>> ---
>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> +++
>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
>>>> ClusterScheduler, actorSystem: Actor
>>>>        case Terminated(actor) =>
>>>>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
>>>> actor terminated"))
>>>>
>>>> -      case DisassociatedEvent(_, remoteAddress, _) =>
>>>> -
>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>> Akka client disconnected"))
>>>> +      // case DisassociatedEvent(_, remoteAddress, _) =>
>>>> +      //
>>>> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>> Akka client disconnected"))
>>>>
>>>>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>>>>
>>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>>> Akka client shutdown"))
>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>> ClusterScheduler, actorSystem: Actor
>>>>      // Remove a disconnected slave from the cluster
>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>        if (executorActor.contains(executorId)) {
>>>> -        logInfo("Executor " + executorId + " disconnected, so removing
>>>> it")
>>>> +        logInfo("Executor " + executorId + " disconnected, so removing
>>>> it, reason:" + reason)
>>>>          val numCores = freeCores(executorId)
>>>>          actorToExecutorId -= executorActor(executorId)
>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>> ok, so I applied a few patches
>>>>>
>>>>> https://github.com/quantifind/incubator-spark/pull/1/files
>>>>>
>>>>> and ran it again, with these options:
>>>>>
>>>>> -Dspark.akka.stdout-loglevel=DEBUG \
>>>>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>>>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>>>>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>>>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>>>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>>>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>>>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>>>>
>>>>> On the driver, I see:
>>>>>
>>>>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO
>>>>> akka.actor.LocalActorRef - Message
>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>> Actor[akka://spark/deadLetters] to
>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>>>>> was not delivered. [1] dead letters encountered. This logging can be turned
>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO
>>>>> org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1
>>>>> disconnected, so removing it, reason:remote Akka client disconnected
>>>>>
>>>>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR
>>>>> org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on
>>>>> dhd2.quantifind.com: remote Akka client disconnected
>>>>>
>>>>>
>>>>> on the worker, stderr:
>>>>>
>>>>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>>>>
>>>>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>>>>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>>>>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>>>>> spark@ddd0.quantifind.com:36730]
>>>>>
>>>>> and unfortunately, all those akka debug options give me *no* useful
>>>>> info in the worker stdout:
>>>>>
>>>>> Starting akka system "sparkExecutor" using config:
>>>>>
>>>>>       akka.daemonic = on
>>>>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>>>>       akka.stdout-loglevel = "DEBUG"
>>>>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>>>>       akka.remote.netty.tcp.transport-class =
>>>>> "akka.remote.transport.netty.NettyTransport"
>>>>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>>>>       akka.remote.netty.tcp.port = 0
>>>>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>>>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>>>>       akka.remote.netty.tcp.execution-pool-size = 4
>>>>>       akka.actor.default-dispatcher.throughput = 15
>>>>>       akka.remote.log-remote-lifecycle-events = off
>>>>>                        akka.remote.log-sent-messages = on
>>>>> akka.remote.log-received-messages = on
>>>>> akka.logLevel = DEBUG
>>>>> akka.actor.debug.autoreceive = on
>>>>> akka.actor.debug.lifecycle = on
>>>>> akka.actor.debug.receive = on
>>>>> akka.log-config-on-start = on
>>>>> akka.remote.quarantine-systems-for = off
>>>>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream]
>>>>> StandardOutLogger started
>>>>> [DEBUG] [10/30/2013 08:40:30.438]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>>>>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>>>>> [DEBUG] [10/30/2013 08:40:30.446]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>>>>> [DEBUG] [10/30/2013 08:40:30.447]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>> [akka://sparkExecutor/system] started
>>>>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>> supervising Actor[akka://sparkExecutor/user]
>>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>>> supervising Actor[akka://sparkExecutor/system]
>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>>> now monitoring Actor[akka://sparkExecutor/system]
>>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>>>>> [DEBUG] [10/30/2013 08:40:30.476]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-3]
>>>>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>>>>> (akka.event.slf4j.Slf4jLogger@24988707)
>>>>> [DEBUG] [10/30/2013 08:40:30.477]
>>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>>> [akka://sparkExecutor/system] now supervising
>>>>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>>>>
>>>>> (followed by similar mesages for the "spark" system)
>>>>>
>>>>> I dunno if this means much more to you, but it seems to me that for
>>>>> some reason the executor decides to disconnect from the master --
>>>>> unfortunately we don't know why.  I think my logging configuration is not
>>>>> getting applied correctly, or "log-sent-messages" & "log-received-messages"
>>>>> don't do what I think they do ... something conflicting must be turing that
>>>>> logging off.  There are a zillion different remoting settings:
>>>>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>>>>
>>>>> I feel like I really need to get those messages on why it disconnected
>>>>> to know which ones to play with.  Any ideas for config changes to see those
>>>>> messages?
>>>>>
>>>>> thanks
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <
>>>>> scrapcodes@gmail.com> wrote:
>>>>>
>>>>>> Can you apply this patch too and check the logs of Driver and worker.
>>>>>>
>>>>>> diff --git
>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> index b6f0ec9..ad0ebf7 100644
>>>>>> ---
>>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> +++
>>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>>> ClusterScheduler, actorSystem: Actor
>>>>>>      // Remove a disconnected slave from the cluster
>>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>>        if (executorActor.contains(executorId)) {
>>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>>> removing it")
>>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>>> removing it, reason:" + reason)
>>>>>>          val numCores = freeCores(executorId)
>>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>
>>>>>>> I just realized something about the failing stages -- they generally
>>>>>>> occur in steps like this:
>>>>>>>
>>>>>>> rdd.mapPartitions{itr =>
>>>>>>>   val myCounters = initializeSomeDataStructure()
>>>>>>>   itr.foreach{
>>>>>>>     //update myCounter in here
>>>>>>>     ...
>>>>>>>   }
>>>>>>>
>>>>>>>   myCounters.iterator.map{
>>>>>>>     //some other transformation here ...
>>>>>>>   }
>>>>>>> }
>>>>>>>
>>>>>>> that is, as a partition is processed, nothing gets output, we just
>>>>>>> accumulate some values.  Only at the end of the partition do we output some
>>>>>>> accumulated values.
>>>>>>>
>>>>>>> These stages don't always fail, and generally they do succeed after
>>>>>>> the executor has died and a new one has started -- so I'm pretty confident
>>>>>>> its not a problem w/ the code.  But maybe we need to add something like a
>>>>>>> periodic heartbeat in this kind of operation?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>>
>>>>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>>>>> http://akka.io/faq/
>>>>>>>> and
>>>>>>>>
>>>>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>>>>
>>>>>>>> unfortunately that will require a patch to spark, but hopefully
>>>>>>>> that will give us more info to go on ...
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <
>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours
>>>>>>>>> now without a problem and my jobs write data about the same as you
>>>>>>>>> suggested. My cluster size is 7 nodes and not *congested* for memory. I
>>>>>>>>> going to leave jobs running all night long. Meanwhile I had encourage you
>>>>>>>>> to try to spot the problem such that it is reproducible that can help a ton
>>>>>>>>> in fixing the issue.
>>>>>>>>>
>>>>>>>>> Thanks for testing and reporting your experience. I still feel
>>>>>>>>> there is something else wrong !. About tolerance for network connection
>>>>>>>>> timeouts, setting those properties should work, but I am afraid about
>>>>>>>>> Disassociation Event though. I will have to check this is indeed hard to
>>>>>>>>> reproduce bug if it is, I mean how do I simulate network delays ?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <
>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>
>>>>>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>>>>>
>>>>>>>>>> At first I thought it must be some temporary network problem too
>>>>>>>>>> -- but the times between receiving task completion events from an executor
>>>>>>>>>> and declaring it failed are really small, so I didn't think that could
>>>>>>>>>> possibly be it.  Plus we tried increasing various akka timeouts, but that
>>>>>>>>>> didn't help.  Or maybe there are some other spark / akka properities we
>>>>>>>>>> should be setting?  It certainly should be resilient to such a temporary
>>>>>>>>>> network issue, if that is the problem.
>>>>>>>>>>
>>>>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>>>>> total for the entire stage).
>>>>>>>>>>
>>>>>>>>>> thanks
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Are you using mesos ? I admit to have not properly tested things
>>>>>>>>>>> on mesos though.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually seen
>>>>>>>>>>>> when a node is disassociated with other by either a network failure or even
>>>>>>>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>>>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>>>>>>>
>>>>>>>>>>>> I had like to have more details on the long jobs though, how
>>>>>>>>>>>> long ?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 22013-10-29 11:03:54,577
>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>> Message
>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2013-10-29 11:03:54,579
>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>> Message [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>>>>> new one is started:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2013-10-29 11:03:58,775
>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-3] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>>> Message
>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not
>>>>>>>>>>>>> delivered. [10] dead letters encountered, no more dead letters will be
>>>>>>>>>>>>> logged. This logging can be turned off or adjusted with configuration
>>>>>>>>>>>>> settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2013-10-29 11:03:58,778
>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter -
>>>>>>>>>>>>> AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068]
>>>>>>>>>>>>> -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]:
>>>>>>>>>>>>> Error [Association failed with [akka.tcp://
>>>>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed
>>>>>>>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>>>>> Caused by:
>>>>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>>>>>
>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>>> [1] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>>> [2] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>>> [3] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>>> [4] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>>>>> while the others sit around.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have no idea what is triggering this behavior -- there isn't
>>>>>>>>>>>>> any particular point in the job that it regularly occurs at.  Certain steps
>>>>>>>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>>>>
>>>>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>>>>
>>>>>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>>>>
>>>>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>>>>>
>>>>>>>>>>>>> thanks,
>>>>>>>>>>>>> Imran
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> --
>>>>>>>>>>>> s
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> s
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> s
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> s
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> s
>>>>
>>>
>>>
>>
>>
>> --
>> s
>>
>
>
>
> --
> s
>

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Hi Imran,

So now my jobs are running for more than 18 hours and there is not a single
executor failure, I can keep it running for more but I don't think that is
going to help. So either you have to give me a way to reproduce the issue
(Which would be great !! ) or you have to diagnose for yourself what
exactly is causing it, a disconnection usually result in Disassociation
event on which we are terminating the executor. In a normal operation this
usually means that driver has finished and those log messages are totally
fine for that matter. I am not sure why you would not see such a thing in
0.8 release and in scala 2.10, can't say anything until I can see what is
happening.



On Thu, Oct 31, 2013 at 4:41 AM, Prashant Sharma <sc...@gmail.com>wrote:

> Can you show us the sample job ? do you do sc.stop at the end or
> System.exit ? Try sc.stop too..
>
>
> On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>
>> yeah, just causes them to hang.
>>
>> the first "deadLetters" message shows up about the same time.  Oddly,
>> after it first happens, I keep getting some results trickling in from those
>> executors.  (maybe they were just queued up on the driver already, I
>> dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
>> but the executors are just idle, they're not running anything.
>>
>> I'm gonna try manually listening for more Association events listed here
>> & logging them
>> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
>>
>> imran
>>
>>
>>
>>
>> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> I am guessing something wrong with using Dissociation event then.
>>>
>>> Try applying something on the lines of this patch. This might cause the
>>> executors to hang so be prepared for that.
>>>
>>> diff --git
>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>> index 4e8052a..1ec5d19 100644
>>> ---
>>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>> +++
>>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>>> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>>>          executor.launchTask(this, taskDesc.taskId,
>>> taskDesc.serializedTask)
>>>        }
>>>
>>> -    case DisassociatedEvent(_, _, _) =>
>>> -      logError("Driver terminated or disconnected! Shutting down.")
>>> +    case Terminated(actor) =>
>>> +      logError("Driver terminated Shutting down.")
>>>        System.exit(1)
>>> +
>>> +    // case DisassociatedEvent(_, _, _) =>
>>> +    //   logError("Driver terminated or disconnected! Shutting down.")
>>> +    //   System.exit(1)
>>>    }
>>>
>>>    override def statusUpdate(taskId: Long, state: TaskState, data:
>>> ByteBuffer) {
>>> diff --git
>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> index b6f0ec9..9955484 100644
>>> ---
>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> +++
>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
>>> ClusterScheduler, actorSystem: Actor
>>>        case Terminated(actor) =>
>>>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
>>> actor terminated"))
>>>
>>> -      case DisassociatedEvent(_, remoteAddress, _) =>
>>> -
>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>> Akka client disconnected"))
>>> +      // case DisassociatedEvent(_, remoteAddress, _) =>
>>> +      //
>>> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>> Akka client disconnected"))
>>>
>>>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>>>
>>>  addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>>> Akka client shutdown"))
>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>> ClusterScheduler, actorSystem: Actor
>>>      // Remove a disconnected slave from the cluster
>>>      def removeExecutor(executorId: String, reason: String) {
>>>        if (executorActor.contains(executorId)) {
>>> -        logInfo("Executor " + executorId + " disconnected, so removing
>>> it")
>>> +        logInfo("Executor " + executorId + " disconnected, so removing
>>> it, reason:" + reason)
>>>          val numCores = freeCores(executorId)
>>>          actorToExecutorId -= executorActor(executorId)
>>>          addressToExecutorId -= executorAddress(executorId)
>>>
>>>
>>>
>>> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> ok, so I applied a few patches
>>>>
>>>> https://github.com/quantifind/incubator-spark/pull/1/files
>>>>
>>>> and ran it again, with these options:
>>>>
>>>> -Dspark.akka.stdout-loglevel=DEBUG \
>>>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>>>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>>>
>>>> On the driver, I see:
>>>>
>>>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO
>>>> akka.actor.LocalActorRef - Message
>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>> Actor[akka://spark/deadLetters] to
>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>>>> was not delivered. [1] dead letters encountered. This logging can be turned
>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO
>>>> org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1
>>>> disconnected, so removing it, reason:remote Akka client disconnected
>>>>
>>>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR
>>>> org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on
>>>> dhd2.quantifind.com: remote Akka client disconnected
>>>>
>>>>
>>>> on the worker, stderr:
>>>>
>>>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>>>
>>>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>>>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>>>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>>>> spark@ddd0.quantifind.com:36730]
>>>>
>>>> and unfortunately, all those akka debug options give me *no* useful
>>>> info in the worker stdout:
>>>>
>>>> Starting akka system "sparkExecutor" using config:
>>>>
>>>>       akka.daemonic = on
>>>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>>>       akka.stdout-loglevel = "DEBUG"
>>>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>>>       akka.remote.netty.tcp.transport-class =
>>>> "akka.remote.transport.netty.NettyTransport"
>>>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>>>       akka.remote.netty.tcp.port = 0
>>>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>>>       akka.remote.netty.tcp.execution-pool-size = 4
>>>>       akka.actor.default-dispatcher.throughput = 15
>>>>       akka.remote.log-remote-lifecycle-events = off
>>>>                        akka.remote.log-sent-messages = on
>>>> akka.remote.log-received-messages = on
>>>> akka.logLevel = DEBUG
>>>> akka.actor.debug.autoreceive = on
>>>> akka.actor.debug.lifecycle = on
>>>> akka.actor.debug.receive = on
>>>> akka.log-config-on-start = on
>>>> akka.remote.quarantine-systems-for = off
>>>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream]
>>>> StandardOutLogger started
>>>> [DEBUG] [10/30/2013 08:40:30.438]
>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>>>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>>>> [DEBUG] [10/30/2013 08:40:30.446]
>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>>>> [DEBUG] [10/30/2013 08:40:30.447]
>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>> [akka://sparkExecutor/system] started
>>>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>> supervising Actor[akka://sparkExecutor/user]
>>>> [DEBUG] [10/30/2013 08:40:30.454]
>>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>>> supervising Actor[akka://sparkExecutor/system]
>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>>> now monitoring Actor[akka://sparkExecutor/system]
>>>> [DEBUG] [10/30/2013 08:40:30.468]
>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>>>> [DEBUG] [10/30/2013 08:40:30.476]
>>>> [sparkExecutor-akka.actor.default-dispatcher-3]
>>>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>>>> (akka.event.slf4j.Slf4jLogger@24988707)
>>>> [DEBUG] [10/30/2013 08:40:30.477]
>>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>>> [akka://sparkExecutor/system] now supervising
>>>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>>>
>>>> (followed by similar mesages for the "spark" system)
>>>>
>>>> I dunno if this means much more to you, but it seems to me that for
>>>> some reason the executor decides to disconnect from the master --
>>>> unfortunately we don't know why.  I think my logging configuration is not
>>>> getting applied correctly, or "log-sent-messages" & "log-received-messages"
>>>> don't do what I think they do ... something conflicting must be turing that
>>>> logging off.  There are a zillion different remoting settings:
>>>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>>>
>>>> I feel like I really need to get those messages on why it disconnected
>>>> to know which ones to play with.  Any ideas for config changes to see those
>>>> messages?
>>>>
>>>> thanks
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <scrapcodes@gmail.com
>>>> > wrote:
>>>>
>>>>> Can you apply this patch too and check the logs of Driver and worker.
>>>>>
>>>>> diff --git
>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> index b6f0ec9..ad0ebf7 100644
>>>>> ---
>>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> +++
>>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>>> ClusterScheduler, actorSystem: Actor
>>>>>      // Remove a disconnected slave from the cluster
>>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>>        if (executorActor.contains(executorId)) {
>>>>> -        logInfo("Executor " + executorId + " disconnected, so
>>>>> removing it")
>>>>> +        logInfo("Executor " + executorId + " disconnected, so
>>>>> removing it, reason:" + reason)
>>>>>          val numCores = freeCores(executorId)
>>>>>          actorToExecutorId -= executorActor(executorId)
>>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>> I just realized something about the failing stages -- they generally
>>>>>> occur in steps like this:
>>>>>>
>>>>>> rdd.mapPartitions{itr =>
>>>>>>   val myCounters = initializeSomeDataStructure()
>>>>>>   itr.foreach{
>>>>>>     //update myCounter in here
>>>>>>     ...
>>>>>>   }
>>>>>>
>>>>>>   myCounters.iterator.map{
>>>>>>     //some other transformation here ...
>>>>>>   }
>>>>>> }
>>>>>>
>>>>>> that is, as a partition is processed, nothing gets output, we just
>>>>>> accumulate some values.  Only at the end of the partition do we output some
>>>>>> accumulated values.
>>>>>>
>>>>>> These stages don't always fail, and generally they do succeed after
>>>>>> the executor has died and a new one has started -- so I'm pretty confident
>>>>>> its not a problem w/ the code.  But maybe we need to add something like a
>>>>>> periodic heartbeat in this kind of operation?
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>
>>>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>>>> http://akka.io/faq/
>>>>>>> and
>>>>>>>
>>>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>>>
>>>>>>> unfortunately that will require a patch to spark, but hopefully that
>>>>>>> will give us more info to go on ...
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <
>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>
>>>>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours
>>>>>>>> now without a problem and my jobs write data about the same as you
>>>>>>>> suggested. My cluster size is 7 nodes and not *congested* for memory. I
>>>>>>>> going to leave jobs running all night long. Meanwhile I had encourage you
>>>>>>>> to try to spot the problem such that it is reproducible that can help a ton
>>>>>>>> in fixing the issue.
>>>>>>>>
>>>>>>>> Thanks for testing and reporting your experience. I still feel
>>>>>>>> there is something else wrong !. About tolerance for network connection
>>>>>>>> timeouts, setting those properties should work, but I am afraid about
>>>>>>>> Disassociation Event though. I will have to check this is indeed hard to
>>>>>>>> reproduce bug if it is, I mean how do I simulate network delays ?
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <imran@quantifind.com
>>>>>>>> > wrote:
>>>>>>>>
>>>>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>>>>
>>>>>>>>> At first I thought it must be some temporary network problem too
>>>>>>>>> -- but the times between receiving task completion events from an executor
>>>>>>>>> and declaring it failed are really small, so I didn't think that could
>>>>>>>>> possibly be it.  Plus we tried increasing various akka timeouts, but that
>>>>>>>>> didn't help.  Or maybe there are some other spark / akka properities we
>>>>>>>>> should be setting?  It certainly should be resilient to such a temporary
>>>>>>>>> network issue, if that is the problem.
>>>>>>>>>
>>>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>>>> total for the entire stage).
>>>>>>>>>
>>>>>>>>> thanks
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>> Are you using mesos ? I admit to have not properly tested things
>>>>>>>>>> on mesos though.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually seen
>>>>>>>>>>> when a node is disassociated with other by either a network failure or even
>>>>>>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>>>>>>
>>>>>>>>>>> I had like to have more details on the long jobs though, how
>>>>>>>>>>> long ?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>>>>
>>>>>>>>>>>> 22013-10-29 11:03:54,577
>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>> Message
>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>> 2013-10-29 11:03:54,579
>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-19] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>>> Message [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>>>> new one is started:
>>>>>>>>>>>>
>>>>>>>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3]
>>>>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered.
>>>>>>>>>>>> [10] dead letters encountered, no more dead letters will be logged. This
>>>>>>>>>>>> logging can be turned off or adjusted with configuration settings
>>>>>>>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>> 2013-10-29 11:03:58,778
>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>>>
>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO
>>>>>>>>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>>>
>>>>>>>>>>>> 2013-10-29 11:03:58,784
>>>>>>>>>>>> [spark-akka.actor.default-dispatcher-18] ERROR akka.remote.EndpointWriter -
>>>>>>>>>>>> AssociationError [akka.tcp://spark@ddd0.quantifind.com:43068]
>>>>>>>>>>>> -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]: Error
>>>>>>>>>>>> [Association failed with [akka.tcp://
>>>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed
>>>>>>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>>>> Caused by:
>>>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>>>
>>>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>>>>
>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>> [1] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>> [2] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>> [3] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>>>
>>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>>> [4] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>>>> while the others sit around.
>>>>>>>>>>>>
>>>>>>>>>>>> I have no idea what is triggering this behavior -- there isn't
>>>>>>>>>>>> any particular point in the job that it regularly occurs at.  Certain steps
>>>>>>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>>>
>>>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>>>
>>>>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>>>
>>>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>>>>
>>>>>>>>>>>> thanks,
>>>>>>>>>>>> Imran
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> s
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> s
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> s
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> s
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>
>
> --
> s
>



-- 
s

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Can you show us the sample job ? do you do sc.stop at the end or
System.exit ? Try sc.stop too..


On Wed, Oct 30, 2013 at 10:42 PM, Imran Rashid <im...@quantifind.com> wrote:

> yeah, just causes them to hang.
>
> the first "deadLetters" message shows up about the same time.  Oddly,
> after it first happens, I keep getting some results trickling in from those
> executors.  (maybe they were just queued up on the driver already, I
> dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
> but the executors are just idle, they're not running anything.
>
> I'm gonna try manually listening for more Association events listed here &
> logging them
> http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events
>
> imran
>
>
>
>
> On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> I am guessing something wrong with using Dissociation event then.
>>
>> Try applying something on the lines of this patch. This might cause the
>> executors to hang so be prepared for that.
>>
>> diff --git
>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>> index 4e8052a..1ec5d19 100644
>> ---
>> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>> +++
>> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
>> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>>          executor.launchTask(this, taskDesc.taskId,
>> taskDesc.serializedTask)
>>        }
>>
>> -    case DisassociatedEvent(_, _, _) =>
>> -      logError("Driver terminated or disconnected! Shutting down.")
>> +    case Terminated(actor) =>
>> +      logError("Driver terminated Shutting down.")
>>        System.exit(1)
>> +
>> +    // case DisassociatedEvent(_, _, _) =>
>> +    //   logError("Driver terminated or disconnected! Shutting down.")
>> +    //   System.exit(1)
>>    }
>>
>>    override def statusUpdate(taskId: Long, state: TaskState, data:
>> ByteBuffer) {
>> diff --git
>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> index b6f0ec9..9955484 100644
>> ---
>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> +++
>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
>> ClusterScheduler, actorSystem: Actor
>>        case Terminated(actor) =>
>>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
>> actor terminated"))
>>
>> -      case DisassociatedEvent(_, remoteAddress, _) =>
>> -        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_,
>> "remote Akka client disconnected"))
>> +      // case DisassociatedEvent(_, remoteAddress, _) =>
>> +      //
>> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
>> Akka client disconnected"))
>>
>>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>>          addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_,
>> "remote Akka client shutdown"))
>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>> ClusterScheduler, actorSystem: Actor
>>      // Remove a disconnected slave from the cluster
>>      def removeExecutor(executorId: String, reason: String) {
>>        if (executorActor.contains(executorId)) {
>> -        logInfo("Executor " + executorId + " disconnected, so removing
>> it")
>> +        logInfo("Executor " + executorId + " disconnected, so removing
>> it, reason:" + reason)
>>          val numCores = freeCores(executorId)
>>          actorToExecutorId -= executorActor(executorId)
>>          addressToExecutorId -= executorAddress(executorId)
>>
>>
>>
>> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> ok, so I applied a few patches
>>>
>>> https://github.com/quantifind/incubator-spark/pull/1/files
>>>
>>> and ran it again, with these options:
>>>
>>> -Dspark.akka.stdout-loglevel=DEBUG \
>>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>>
>>> On the driver, I see:
>>>
>>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO
>>> akka.actor.LocalActorRef - Message
>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>> Actor[akka://spark/deadLetters] to
>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>>> was not delivered. [1] dead letters encountered. This logging can be turned
>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO
>>> org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1
>>> disconnected, so removing it, reason:remote Akka client disconnected
>>>
>>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR
>>> org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on
>>> dhd2.quantifind.com: remote Akka client disconnected
>>>
>>>
>>> on the worker, stderr:
>>>
>>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>>
>>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>>> spark@ddd0.quantifind.com:36730]
>>>
>>> and unfortunately, all those akka debug options give me *no* useful info
>>> in the worker stdout:
>>>
>>> Starting akka system "sparkExecutor" using config:
>>>
>>>       akka.daemonic = on
>>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>>       akka.stdout-loglevel = "DEBUG"
>>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>>       akka.remote.netty.tcp.transport-class =
>>> "akka.remote.transport.netty.NettyTransport"
>>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>>       akka.remote.netty.tcp.port = 0
>>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>>       akka.remote.netty.tcp.execution-pool-size = 4
>>>       akka.actor.default-dispatcher.throughput = 15
>>>       akka.remote.log-remote-lifecycle-events = off
>>>                        akka.remote.log-sent-messages = on
>>> akka.remote.log-received-messages = on
>>> akka.logLevel = DEBUG
>>> akka.actor.debug.autoreceive = on
>>> akka.actor.debug.lifecycle = on
>>> akka.actor.debug.receive = on
>>> akka.log-config-on-start = on
>>> akka.remote.quarantine-systems-for = off
>>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream] StandardOutLogger
>>> started
>>> [DEBUG] [10/30/2013 08:40:30.438]
>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>>> [DEBUG] [10/30/2013 08:40:30.446]
>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>>> [DEBUG] [10/30/2013 08:40:30.447]
>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>> [akka://sparkExecutor/system] started
>>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>>> [DEBUG] [10/30/2013 08:40:30.454]
>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>> supervising Actor[akka://sparkExecutor/user]
>>> [DEBUG] [10/30/2013 08:40:30.454]
>>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>>> supervising Actor[akka://sparkExecutor/system]
>>> [DEBUG] [10/30/2013 08:40:30.468]
>>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>>> now monitoring Actor[akka://sparkExecutor/system]
>>> [DEBUG] [10/30/2013 08:40:30.468]
>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>>> [DEBUG] [10/30/2013 08:40:30.476]
>>> [sparkExecutor-akka.actor.default-dispatcher-3]
>>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>>> (akka.event.slf4j.Slf4jLogger@24988707)
>>> [DEBUG] [10/30/2013 08:40:30.477]
>>> [sparkExecutor-akka.actor.default-dispatcher-4]
>>> [akka://sparkExecutor/system] now supervising
>>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>>
>>> (followed by similar mesages for the "spark" system)
>>>
>>> I dunno if this means much more to you, but it seems to me that for some
>>> reason the executor decides to disconnect from the master -- unfortunately
>>> we don't know why.  I think my logging configuration is not getting applied
>>> correctly, or "log-sent-messages" & "log-received-messages" don't do what I
>>> think they do ... something conflicting must be turing that logging off.
>>> There are a zillion different remoting settings:
>>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>>
>>> I feel like I really need to get those messages on why it disconnected
>>> to know which ones to play with.  Any ideas for config changes to see those
>>> messages?
>>>
>>> thanks
>>>
>>>
>>>
>>>
>>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>
>>>> Can you apply this patch too and check the logs of Driver and worker.
>>>>
>>>> diff --git
>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> index b6f0ec9..ad0ebf7 100644
>>>> ---
>>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> +++
>>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>>> ClusterScheduler, actorSystem: Actor
>>>>      // Remove a disconnected slave from the cluster
>>>>      def removeExecutor(executorId: String, reason: String) {
>>>>        if (executorActor.contains(executorId)) {
>>>> -        logInfo("Executor " + executorId + " disconnected, so removing
>>>> it")
>>>> +        logInfo("Executor " + executorId + " disconnected, so removing
>>>> it, reason:" + reason)
>>>>          val numCores = freeCores(executorId)
>>>>          actorToExecutorId -= executorActor(executorId)
>>>>          addressToExecutorId -= executorAddress(executorId)
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>> I just realized something about the failing stages -- they generally
>>>>> occur in steps like this:
>>>>>
>>>>> rdd.mapPartitions{itr =>
>>>>>   val myCounters = initializeSomeDataStructure()
>>>>>   itr.foreach{
>>>>>     //update myCounter in here
>>>>>     ...
>>>>>   }
>>>>>
>>>>>   myCounters.iterator.map{
>>>>>     //some other transformation here ...
>>>>>   }
>>>>> }
>>>>>
>>>>> that is, as a partition is processed, nothing gets output, we just
>>>>> accumulate some values.  Only at the end of the partition do we output some
>>>>> accumulated values.
>>>>>
>>>>> These stages don't always fail, and generally they do succeed after
>>>>> the executor has died and a new one has started -- so I'm pretty confident
>>>>> its not a problem w/ the code.  But maybe we need to add something like a
>>>>> periodic heartbeat in this kind of operation?
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>>> http://akka.io/faq/
>>>>>> and
>>>>>>
>>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>>
>>>>>> unfortunately that will require a patch to spark, but hopefully that
>>>>>> will give us more info to go on ...
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <
>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>
>>>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours
>>>>>>> now without a problem and my jobs write data about the same as you
>>>>>>> suggested. My cluster size is 7 nodes and not *congested* for memory. I
>>>>>>> going to leave jobs running all night long. Meanwhile I had encourage you
>>>>>>> to try to spot the problem such that it is reproducible that can help a ton
>>>>>>> in fixing the issue.
>>>>>>>
>>>>>>> Thanks for testing and reporting your experience. I still feel there
>>>>>>> is something else wrong !. About tolerance for network connection timeouts,
>>>>>>> setting those properties should work, but I am afraid about Disassociation
>>>>>>> Event though. I will have to check this is indeed hard to reproduce bug if
>>>>>>> it is, I mean how do I simulate network delays ?
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>>
>>>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>>>
>>>>>>>> At first I thought it must be some temporary network problem too --
>>>>>>>> but the times between receiving task completion events from an executor and
>>>>>>>> declaring it failed are really small, so I didn't think that could possibly
>>>>>>>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>>>>>>>> help.  Or maybe there are some other spark / akka properities we should be
>>>>>>>> setting?  It certainly should be resilient to such a temporary network
>>>>>>>> issue, if that is the problem.
>>>>>>>>
>>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>>> total for the entire stage).
>>>>>>>>
>>>>>>>> thanks
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> Are you using mesos ? I admit to have not properly tested things
>>>>>>>>> on mesos though.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually seen
>>>>>>>>>> when a node is disassociated with other by either a network failure or even
>>>>>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>>>>>
>>>>>>>>>> I had like to have more details on the long jobs though, how long
>>>>>>>>>> ?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>>>
>>>>>>>>>>> 22013-10-29 11:03:54,577
>>>>>>>>>>> [spark-akka.actor.default-dispatcher-17] INFO  akka.actor.LocalActorRef -
>>>>>>>>>>> Message
>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19]
>>>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>>> new one is started:
>>>>>>>>>>>
>>>>>>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3]
>>>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered.
>>>>>>>>>>> [10] dead letters encountered, no more dead letters will be logged. This
>>>>>>>>>>> logging can be turned off or adjusted with configuration settings
>>>>>>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>>
>>>>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>>
>>>>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18]
>>>>>>>>>>> ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>>>>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association
>>>>>>>>>>> failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]]
>>>>>>>>>>> [
>>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed
>>>>>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>>> Caused by:
>>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>>
>>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>>>
>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>> [1] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>> [2] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>> [3] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>>
>>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>>> [4] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>>> while the others sit around.
>>>>>>>>>>>
>>>>>>>>>>> I have no idea what is triggering this behavior -- there isn't
>>>>>>>>>>> any particular point in the job that it regularly occurs at.  Certain steps
>>>>>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>>
>>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>>
>>>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>>
>>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>>>
>>>>>>>>>>> thanks,
>>>>>>>>>>> Imran
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> s
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> s
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> s
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> s
>>>>
>>>
>>>
>>
>>
>> --
>> s
>>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
yeah, just causes them to hang.

the first "deadLetters" message shows up about the same time.  Oddly, after
it first happens, I keep getting some results trickling in from those
executors.  (maybe they were just queued up on the driver already, I
dunno.)  but then it just hangs.  the stage has a few more tasks to be run,
but the executors are just idle, they're not running anything.

I'm gonna try manually listening for more Association events listed here &
logging them
http://doc.akka.io/docs/akka/2.2.3/scala/remoting.html#remote-events

imran




On Wed, Oct 30, 2013 at 11:27 AM, Prashant Sharma <sc...@gmail.com>wrote:

> I am guessing something wrong with using Dissociation event then.
>
> Try applying something on the lines of this patch. This might cause the
> executors to hang so be prepared for that.
>
> diff --git
> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> index 4e8052a..1ec5d19 100644
> ---
> a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> +++
> b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
> @@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
>          executor.launchTask(this, taskDesc.taskId,
> taskDesc.serializedTask)
>        }
>
> -    case DisassociatedEvent(_, _, _) =>
> -      logError("Driver terminated or disconnected! Shutting down.")
> +    case Terminated(actor) =>
> +      logError("Driver terminated Shutting down.")
>        System.exit(1)
> +
> +    // case DisassociatedEvent(_, _, _) =>
> +    //   logError("Driver terminated or disconnected! Shutting down.")
> +    //   System.exit(1)
>    }
>
>    override def statusUpdate(taskId: Long, state: TaskState, data:
> ByteBuffer) {
> diff --git
> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> index b6f0ec9..9955484 100644
> ---
> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> +++
> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> @@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
> ClusterScheduler, actorSystem: Actor
>        case Terminated(actor) =>
>          actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka
> actor terminated"))
>
> -      case DisassociatedEvent(_, remoteAddress, _) =>
> -        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_,
> "remote Akka client disconnected"))
> +      // case DisassociatedEvent(_, remoteAddress, _) =>
> +      //
> addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
> Akka client disconnected"))
>
>        case AssociationErrorEvent(_, _, remoteAddress, _) =>
>          addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_,
> "remote Akka client shutdown"))
> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
> ClusterScheduler, actorSystem: Actor
>      // Remove a disconnected slave from the cluster
>      def removeExecutor(executorId: String, reason: String) {
>        if (executorActor.contains(executorId)) {
> -        logInfo("Executor " + executorId + " disconnected, so removing
> it")
> +        logInfo("Executor " + executorId + " disconnected, so removing
> it, reason:" + reason)
>          val numCores = freeCores(executorId)
>          actorToExecutorId -= executorActor(executorId)
>          addressToExecutorId -= executorAddress(executorId)
>
>
>
> On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com>wrote:
>
>> ok, so I applied a few patches
>>
>> https://github.com/quantifind/incubator-spark/pull/1/files
>>
>> and ran it again, with these options:
>>
>> -Dspark.akka.stdout-loglevel=DEBUG \
>>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
>> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>>   -Dspark.akkaExtra.akka.log-config-on-start=on
>>
>> On the driver, I see:
>>
>> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO
>> akka.actor.LocalActorRef - Message
>> [akka.remote.transport.AssociationHandle$Disassociated] from
>> Actor[akka://spark/deadLetters] to
>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
>> was not delivered. [1] dead letters encountered. This logging can be turned
>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO
>> org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1
>> disconnected, so removing it, reason:remote Akka client disconnected
>>
>> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR
>> org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on
>> dhd2.quantifind.com: remote Akka client disconnected
>>
>>
>> on the worker, stderr:
>>
>> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>>
>> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
>> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
>> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
>> spark@ddd0.quantifind.com:36730]
>>
>> and unfortunately, all those akka debug options give me *no* useful info
>> in the worker stdout:
>>
>> Starting akka system "sparkExecutor" using config:
>>
>>       akka.daemonic = on
>>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>>       akka.stdout-loglevel = "DEBUG"
>>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>>       akka.remote.netty.tcp.transport-class =
>> "akka.remote.transport.netty.NettyTransport"
>>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>>       akka.remote.netty.tcp.port = 0
>>       akka.remote.netty.tcp.connection-timeout = 60 s
>>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>>       akka.remote.netty.tcp.execution-pool-size = 4
>>       akka.actor.default-dispatcher.throughput = 15
>>       akka.remote.log-remote-lifecycle-events = off
>>                        akka.remote.log-sent-messages = on
>> akka.remote.log-received-messages = on
>> akka.logLevel = DEBUG
>> akka.actor.debug.autoreceive = on
>> akka.actor.debug.lifecycle = on
>> akka.actor.debug.receive = on
>> akka.log-config-on-start = on
>> akka.remote.quarantine-systems-for = off
>> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream] StandardOutLogger
>> started
>> [DEBUG] [10/30/2013 08:40:30.438]
>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
>> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
>> [DEBUG] [10/30/2013 08:40:30.446]
>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
>> [DEBUG] [10/30/2013 08:40:30.447]
>> [sparkExecutor-akka.actor.default-dispatcher-4]
>> [akka://sparkExecutor/system] started
>> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
>> [DEBUG] [10/30/2013 08:40:30.454]
>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>> supervising Actor[akka://sparkExecutor/user]
>> [DEBUG] [10/30/2013 08:40:30.454]
>> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
>> supervising Actor[akka://sparkExecutor/system]
>> [DEBUG] [10/30/2013 08:40:30.468]
>> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
>> now monitoring Actor[akka://sparkExecutor/system]
>> [DEBUG] [10/30/2013 08:40:30.468]
>> [sparkExecutor-akka.actor.default-dispatcher-4]
>> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
>> [DEBUG] [10/30/2013 08:40:30.476]
>> [sparkExecutor-akka.actor.default-dispatcher-3]
>> [akka://sparkExecutor/system/log1-Slf4jLogger] started
>> (akka.event.slf4j.Slf4jLogger@24988707)
>> [DEBUG] [10/30/2013 08:40:30.477]
>> [sparkExecutor-akka.actor.default-dispatcher-4]
>> [akka://sparkExecutor/system] now supervising
>> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>>
>> (followed by similar mesages for the "spark" system)
>>
>> I dunno if this means much more to you, but it seems to me that for some
>> reason the executor decides to disconnect from the master -- unfortunately
>> we don't know why.  I think my logging configuration is not getting applied
>> correctly, or "log-sent-messages" & "log-received-messages" don't do what I
>> think they do ... something conflicting must be turing that logging off.
>> There are a zillion different remoting settings:
>> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>>
>> I feel like I really need to get those messages on why it disconnected to
>> know which ones to play with.  Any ideas for config changes to see those
>> messages?
>>
>> thanks
>>
>>
>>
>>
>> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> Can you apply this patch too and check the logs of Driver and worker.
>>>
>>> diff --git
>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> index b6f0ec9..ad0ebf7 100644
>>> ---
>>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> +++
>>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>>> ClusterScheduler, actorSystem: Actor
>>>      // Remove a disconnected slave from the cluster
>>>      def removeExecutor(executorId: String, reason: String) {
>>>        if (executorActor.contains(executorId)) {
>>> -        logInfo("Executor " + executorId + " disconnected, so removing
>>> it")
>>> +        logInfo("Executor " + executorId + " disconnected, so removing
>>> it, reason:" + reason)
>>>          val numCores = freeCores(executorId)
>>>          actorToExecutorId -= executorActor(executorId)
>>>          addressToExecutorId -= executorAddress(executorId)
>>>
>>>
>>>
>>>
>>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> I just realized something about the failing stages -- they generally
>>>> occur in steps like this:
>>>>
>>>> rdd.mapPartitions{itr =>
>>>>   val myCounters = initializeSomeDataStructure()
>>>>   itr.foreach{
>>>>     //update myCounter in here
>>>>     ...
>>>>   }
>>>>
>>>>   myCounters.iterator.map{
>>>>     //some other transformation here ...
>>>>   }
>>>> }
>>>>
>>>> that is, as a partition is processed, nothing gets output, we just
>>>> accumulate some values.  Only at the end of the partition do we output some
>>>> accumulated values.
>>>>
>>>> These stages don't always fail, and generally they do succeed after the
>>>> executor has died and a new one has started -- so I'm pretty confident its
>>>> not a problem w/ the code.  But maybe we need to add something like a
>>>> periodic heartbeat in this kind of operation?
>>>>
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>>> http://akka.io/faq/
>>>>> and
>>>>>
>>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>>
>>>>> unfortunately that will require a patch to spark, but hopefully that
>>>>> will give us more info to go on ...
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <scrapcodes@gmail.com
>>>>> > wrote:
>>>>>
>>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours now
>>>>>> without a problem and my jobs write data about the same as you suggested.
>>>>>> My cluster size is 7 nodes and not *congested* for memory. I going to leave
>>>>>> jobs running all night long. Meanwhile I had encourage you to try to spot
>>>>>> the problem such that it is reproducible that can help a ton in fixing the
>>>>>> issue.
>>>>>>
>>>>>> Thanks for testing and reporting your experience. I still feel there
>>>>>> is something else wrong !. About tolerance for network connection timeouts,
>>>>>> setting those properties should work, but I am afraid about Disassociation
>>>>>> Event though. I will have to check this is indeed hard to reproduce bug if
>>>>>> it is, I mean how do I simulate network delays ?
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>
>>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>>
>>>>>>> At first I thought it must be some temporary network problem too --
>>>>>>> but the times between receiving task completion events from an executor and
>>>>>>> declaring it failed are really small, so I didn't think that could possibly
>>>>>>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>>>>>>> help.  Or maybe there are some other spark / akka properities we should be
>>>>>>> setting?  It certainly should be resilient to such a temporary network
>>>>>>> issue, if that is the problem.
>>>>>>>
>>>>>>> btw, I think I've noticed this happens most often during
>>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>>> total for the entire stage).
>>>>>>>
>>>>>>> thanks
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>
>>>>>>>> Are you using mesos ? I admit to have not properly tested things on
>>>>>>>> mesos though.
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> Those log messages are new to the Akka 2.2 and are usually seen
>>>>>>>>> when a node is disassociated with other by either a network failure or even
>>>>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>>>>
>>>>>>>>> I had like to have more details on the long jobs though, how long
>>>>>>>>> ?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <
>>>>>>>>> imran@quantifind.com> wrote:
>>>>>>>>>
>>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're
>>>>>>>>>> running into some issues were akka disconnects from the executors after a
>>>>>>>>>> while.  We ran some simple tests first, and all was well, so we started
>>>>>>>>>> upgrading our whole codebase to 2.10.  Everything seemed to be working, but
>>>>>>>>>> then we noticed that when we run long jobs, and then things start failing.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>>
>>>>>>>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19]
>>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Generally within a few seconds after the first such message,
>>>>>>>>>> there are a bunch more, and then the executor is marked as failed, and a
>>>>>>>>>> new one is started:
>>>>>>>>>>
>>>>>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3]
>>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered.
>>>>>>>>>> [10] dead letters encountered, no more dead letters will be logged. This
>>>>>>>>>> logging can be turned off or adjusted with configuration settings
>>>>>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>>
>>>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>>
>>>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18]
>>>>>>>>>> ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>>>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association
>>>>>>>>>> failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]]
>>>>>>>>>> [
>>>>>>>>>> akka.remote.EndpointAssociationException: Association failed with
>>>>>>>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>>> Caused by:
>>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Looking in the logs of the failed executor, there are some
>>>>>>>>>> similar messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>>
>>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>>
>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>> [1] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>> [2] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>> [3] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend:
>>>>>>>>>> Driver terminated or disconnected! Shutting down.
>>>>>>>>>>
>>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered.
>>>>>>>>>> [4] dead letters encountered. This logging can be turned off or adjusted
>>>>>>>>>> with configuration settings 'akka.log-dead-letters' and
>>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> After this happens, spark does launch a new executor
>>>>>>>>>> successfully, and continue the job.  Sometimes, the job just continues
>>>>>>>>>> happily and there aren't any other problems.  However, that executor may
>>>>>>>>>> have to run a bunch of steps to re-compute some cached RDDs -- and during
>>>>>>>>>> that time, another executor may crash similarly, and then we end up in a
>>>>>>>>>> never ending loop, of one executor crashing, then trying to reload data,
>>>>>>>>>> while the others sit around.
>>>>>>>>>>
>>>>>>>>>> I have no idea what is triggering this behavior -- there isn't
>>>>>>>>>> any particular point in the job that it regularly occurs at.  Certain steps
>>>>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>>
>>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>>
>>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>>
>>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>>
>>>>>>>>>> thanks,
>>>>>>>>>> Imran
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> s
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> s
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> s
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>
>
> --
> s
>

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
I am guessing something wrong with using Dissociation event then.

Try applying something on the lines of this patch. This might cause the
executors to hang so be prepared for that.

diff --git
a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
index 4e8052a..1ec5d19 100644
---
a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
+++
b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
@@ -74,9 +74,13 @@ private[spark] class StandaloneExecutorBackend(
         executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
       }

-    case DisassociatedEvent(_, _, _) =>
-      logError("Driver terminated or disconnected! Shutting down.")
+    case Terminated(actor) =>
+      logError("Driver terminated Shutting down.")
       System.exit(1)
+
+    // case DisassociatedEvent(_, _, _) =>
+    //   logError("Driver terminated or disconnected! Shutting down.")
+    //   System.exit(1)
   }

   override def statusUpdate(taskId: Long, state: TaskState, data:
ByteBuffer) {
diff --git
a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index b6f0ec9..9955484 100644
---
a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++
b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -102,8 +102,8 @@ class StandaloneSchedulerBackend(scheduler:
ClusterScheduler, actorSystem: Actor
       case Terminated(actor) =>
         actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor
terminated"))

-      case DisassociatedEvent(_, remoteAddress, _) =>
-        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_,
"remote Akka client disconnected"))
+      // case DisassociatedEvent(_, remoteAddress, _) =>
+      //
addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote
Akka client disconnected"))

       case AssociationErrorEvent(_, _, remoteAddress, _) =>
         addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_,
"remote Akka client shutdown"))
@@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
ClusterScheduler, actorSystem: Actor
     // Remove a disconnected slave from the cluster
     def removeExecutor(executorId: String, reason: String) {
       if (executorActor.contains(executorId)) {
-        logInfo("Executor " + executorId + " disconnected, so removing it")
+        logInfo("Executor " + executorId + " disconnected, so removing it,
reason:" + reason)
         val numCores = freeCores(executorId)
         actorToExecutorId -= executorActor(executorId)
         addressToExecutorId -= executorAddress(executorId)



On Wed, Oct 30, 2013 at 9:42 PM, Imran Rashid <im...@quantifind.com> wrote:

> ok, so I applied a few patches
>
> https://github.com/quantifind/incubator-spark/pull/1/files
>
> and ran it again, with these options:
>
> -Dspark.akka.stdout-loglevel=DEBUG \
>   -Dspark.akkaExtra.akka.logLevel=DEBUG\
>   -Dspark.akkaExtra.akka.actor.debug.receive=on \
> -Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
>   -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
>   -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
>   -Dspark.akkaExtra.akka.remote.log-received-messages=on\
>   -Dspark.akkaExtra.akka.log-config-on-start=on
>
> On the driver, I see:
>
> 2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO
> akka.actor.LocalActorRef - Message
> [akka.remote.transport.AssociationHandle$Disassociated] from
> Actor[akka://spark/deadLetters] to
> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
> was not delivered. [1] dead letters encountered. This logging can be turned
> off or adjusted with configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
> 2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO
> org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1
> disconnected, so removing it, reason:remote Akka client disconnected
>
> 2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR
> org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on
> dhd2.quantifind.com: remote Akka client disconnected
>
>
> on the worker, stderr:
>
> 13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934
>
> 13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
> terminated or disconnected! Shutting down.Disassociated [akka.tcp://
> sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
> spark@ddd0.quantifind.com:36730]
>
> and unfortunately, all those akka debug options give me *no* useful info
> in the worker stdout:
>
> Starting akka system "sparkExecutor" using config:
>
>       akka.daemonic = on
>       akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
>       akka.stdout-loglevel = "DEBUG"
>       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
>       akka.remote.netty.tcp.transport-class =
> "akka.remote.transport.netty.NettyTransport"
>       akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
>       akka.remote.netty.tcp.port = 0
>       akka.remote.netty.tcp.connection-timeout = 60 s
>       akka.remote.netty.tcp.maximum-frame-size = 10MiB
>       akka.remote.netty.tcp.execution-pool-size = 4
>       akka.actor.default-dispatcher.throughput = 15
>       akka.remote.log-remote-lifecycle-events = off
>                        akka.remote.log-sent-messages = on
> akka.remote.log-received-messages = on
> akka.logLevel = DEBUG
> akka.actor.debug.autoreceive = on
> akka.actor.debug.lifecycle = on
> akka.actor.debug.receive = on
> akka.log-config-on-start = on
> akka.remote.quarantine-systems-for = off
> [DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream] StandardOutLogger
> started
> [DEBUG] [10/30/2013 08:40:30.438]
> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
> started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
> [DEBUG] [10/30/2013 08:40:30.446]
> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
> started (akka.actor.LocalActorRefProvider$Guardian@72608760)
> [DEBUG] [10/30/2013 08:40:30.447]
> [sparkExecutor-akka.actor.default-dispatcher-4]
> [akka://sparkExecutor/system] started
> (akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
> [DEBUG] [10/30/2013 08:40:30.454]
> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
> supervising Actor[akka://sparkExecutor/user]
> [DEBUG] [10/30/2013 08:40:30.454]
> [sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
> supervising Actor[akka://sparkExecutor/system]
> [DEBUG] [10/30/2013 08:40:30.468]
> [sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
> now monitoring Actor[akka://sparkExecutor/system]
> [DEBUG] [10/30/2013 08:40:30.468]
> [sparkExecutor-akka.actor.default-dispatcher-4]
> [akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
> [DEBUG] [10/30/2013 08:40:30.476]
> [sparkExecutor-akka.actor.default-dispatcher-3]
> [akka://sparkExecutor/system/log1-Slf4jLogger] started
> (akka.event.slf4j.Slf4jLogger@24988707)
> [DEBUG] [10/30/2013 08:40:30.477]
> [sparkExecutor-akka.actor.default-dispatcher-4]
> [akka://sparkExecutor/system] now supervising
> Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]
>
> (followed by similar mesages for the "spark" system)
>
> I dunno if this means much more to you, but it seems to me that for some
> reason the executor decides to disconnect from the master -- unfortunately
> we don't know why.  I think my logging configuration is not getting applied
> correctly, or "log-sent-messages" & "log-received-messages" don't do what I
> think they do ... something conflicting must be turing that logging off.
> There are a zillion different remoting settings:
> http://doc.akka.io/docs/akka/snapshot/scala/remoting.html
>
> I feel like I really need to get those messages on why it disconnected to
> know which ones to play with.  Any ideas for config changes to see those
> messages?
>
> thanks
>
>
>
>
> On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> Can you apply this patch too and check the logs of Driver and worker.
>>
>> diff --git
>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> index b6f0ec9..ad0ebf7 100644
>> ---
>> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> +++
>> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
>> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
>> ClusterScheduler, actorSystem: Actor
>>      // Remove a disconnected slave from the cluster
>>      def removeExecutor(executorId: String, reason: String) {
>>        if (executorActor.contains(executorId)) {
>> -        logInfo("Executor " + executorId + " disconnected, so removing
>> it")
>> +        logInfo("Executor " + executorId + " disconnected, so removing
>> it, reason:" + reason)
>>          val numCores = freeCores(executorId)
>>          actorToExecutorId -= executorActor(executorId)
>>          addressToExecutorId -= executorAddress(executorId)
>>
>>
>>
>>
>> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> I just realized something about the failing stages -- they generally
>>> occur in steps like this:
>>>
>>> rdd.mapPartitions{itr =>
>>>   val myCounters = initializeSomeDataStructure()
>>>   itr.foreach{
>>>     //update myCounter in here
>>>     ...
>>>   }
>>>
>>>   myCounters.iterator.map{
>>>     //some other transformation here ...
>>>   }
>>> }
>>>
>>> that is, as a partition is processed, nothing gets output, we just
>>> accumulate some values.  Only at the end of the partition do we output some
>>> accumulated values.
>>>
>>> These stages don't always fail, and generally they do succeed after the
>>> executor has died and a new one has started -- so I'm pretty confident its
>>> not a problem w/ the code.  But maybe we need to add something like a
>>> periodic heartbeat in this kind of operation?
>>>
>>>
>>>
>>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> I'm gonna try turning on more akka debugging msgs as described at
>>>> http://akka.io/faq/
>>>> and
>>>>
>>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>>
>>>> unfortunately that will require a patch to spark, but hopefully that
>>>> will give us more info to go on ...
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>>
>>>>> I have things running (from scala 2.10 branch) for over 3-4 hours now
>>>>> without a problem and my jobs write data about the same as you suggested.
>>>>> My cluster size is 7 nodes and not *congested* for memory. I going to leave
>>>>> jobs running all night long. Meanwhile I had encourage you to try to spot
>>>>> the problem such that it is reproducible that can help a ton in fixing the
>>>>> issue.
>>>>>
>>>>> Thanks for testing and reporting your experience. I still feel there
>>>>> is something else wrong !. About tolerance for network connection timeouts,
>>>>> setting those properties should work, but I am afraid about Disassociation
>>>>> Event though. I will have to check this is indeed hard to reproduce bug if
>>>>> it is, I mean how do I simulate network delays ?
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>>
>>>>>> At first I thought it must be some temporary network problem too --
>>>>>> but the times between receiving task completion events from an executor and
>>>>>> declaring it failed are really small, so I didn't think that could possibly
>>>>>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>>>>>> help.  Or maybe there are some other spark / akka properities we should be
>>>>>> setting?  It certainly should be resilient to such a temporary network
>>>>>> issue, if that is the problem.
>>>>>>
>>>>>> btw, I think I've noticed this happens most often during
>>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>>> total for the entire stage).
>>>>>>
>>>>>> thanks
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <
>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>
>>>>>>> Are you using mesos ? I admit to have not properly tested things on
>>>>>>> mesos though.
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>>
>>>>>>>> Those log messages are new to the Akka 2.2 and are usually seen
>>>>>>>> when a node is disassociated with other by either a network failure or even
>>>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>>>
>>>>>>>> I had like to have more details on the long jobs though, how long ?
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <imran@quantifind.com
>>>>>>>> > wrote:
>>>>>>>>
>>>>>>>>> We've been testing out the 2.10 branch of spark, and we're running
>>>>>>>>> into some issues were akka disconnects from the executors after a while.
>>>>>>>>> We ran some simple tests first, and all was well, so we started upgrading
>>>>>>>>> our whole codebase to 2.10.  Everything seemed to be working, but then we
>>>>>>>>> noticed that when we run long jobs, and then things start failing.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>>
>>>>>>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19]
>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Generally within a few seconds after the first such message, there
>>>>>>>>> are a bunch more, and then the executor is marked as failed, and a new one
>>>>>>>>> is started:
>>>>>>>>>
>>>>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3]
>>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered.
>>>>>>>>> [10] dead letters encountered, no more dead letters will be logged. This
>>>>>>>>> logging can be turned off or adjusted with configuration settings
>>>>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>>
>>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17]
>>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>>
>>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18]
>>>>>>>>> ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association
>>>>>>>>> failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]]
>>>>>>>>> [
>>>>>>>>> akka.remote.EndpointAssociationException: Association failed with
>>>>>>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>>> Caused by:
>>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Looking in the logs of the failed executor, there are some similar
>>>>>>>>> messages about undeliverable messages, but I don't see any reason:
>>>>>>>>>
>>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>>
>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1]
>>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2]
>>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3]
>>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>>>>> terminated or disconnected! Shutting down.
>>>>>>>>>
>>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4]
>>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> After this happens, spark does launch a new executor successfully,
>>>>>>>>> and continue the job.  Sometimes, the job just continues happily and there
>>>>>>>>> aren't any other problems.  However, that executor may have to run a bunch
>>>>>>>>> of steps to re-compute some cached RDDs -- and during that time, another
>>>>>>>>> executor may crash similarly, and then we end up in a never ending loop, of
>>>>>>>>> one executor crashing, then trying to reload data, while the others sit
>>>>>>>>> around.
>>>>>>>>>
>>>>>>>>> I have no idea what is triggering this behavior -- there isn't any
>>>>>>>>> particular point in the job that it regularly occurs at.  Certain steps
>>>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>>
>>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>>
>>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>>
>>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>>
>>>>>>>>> thanks,
>>>>>>>>> Imran
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> s
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> s
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> s
>>>>>
>>>>
>>>>
>>>
>>
>>
>> --
>> s
>>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
ok, so I applied a few patches

https://github.com/quantifind/incubator-spark/pull/1/files

and ran it again, with these options:

-Dspark.akka.stdout-loglevel=DEBUG \
  -Dspark.akkaExtra.akka.logLevel=DEBUG\
  -Dspark.akkaExtra.akka.actor.debug.receive=on \
-Dspark.akkaExtra.akka.actor.debug.autoreceive=on \
  -Dspark.akkaExtra.akka.actor.debug.lifecycle=on \
  -Dspark.akkaExtra.akka.remote.log-sent-messages=on \
  -Dspark.akkaExtra.akka.remote.log-received-messages=on\
  -Dspark.akkaExtra.akka.log-config-on-start=on

On the driver, I see:

2013-10-30 08:44:31,034 [spark-akka.actor.default-dispatcher-19] INFO
akka.actor.LocalActorRef - Message
[akka.remote.transport.AssociationHandle$Disassociated] from
Actor[akka://spark/deadLetters] to
Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.64%3A52400-2#-837892141]
was not delivered. [1] dead letters encountered. This logging can be turned
off or adjusted with configuration settings 'akka.log-dead-letters' and
'akka.log-dead-letters-during-shutdown'.

2013-10-30 08:44:31,058 [spark-akka.actor.default-dispatcher-13] INFO
org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend - Executor 1
disconnected, so removing it, reason:remote Akka client disconnected

2013-10-30 08:44:31,059 [spark-akka.actor.default-dispatcher-13] ERROR
org.apache.spark.scheduler.cluster.ClusterScheduler - Lost executor 1 on
dhd2.quantifind.com: remote Akka client disconnected


on the worker, stderr:

13/10/30 08:44:28 INFO executor.Executor: Finished task ID 934

13/10/30 08:44:31 ERROR executor.StandaloneExecutorBackend: Driver
terminated or disconnected! Shutting down.Disassociated [akka.tcp://
sparkExecutor@dhd2.quantifind.com:38021] -> [akka.tcp://
spark@ddd0.quantifind.com:36730]

and unfortunately, all those akka debug options give me *no* useful info in
the worker stdout:

Starting akka system "sparkExecutor" using config:

      akka.daemonic = on
      akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
      akka.stdout-loglevel = "DEBUG"
      akka.actor.provider = "akka.remote.RemoteActorRefProvider"
      akka.remote.netty.tcp.transport-class =
"akka.remote.transport.netty.NettyTransport"
      akka.remote.netty.tcp.hostname = "dhd2.quantifind.com"
      akka.remote.netty.tcp.port = 0
      akka.remote.netty.tcp.connection-timeout = 60 s
      akka.remote.netty.tcp.maximum-frame-size = 10MiB
      akka.remote.netty.tcp.execution-pool-size = 4
      akka.actor.default-dispatcher.throughput = 15
      akka.remote.log-remote-lifecycle-events = off
                       akka.remote.log-sent-messages = on
akka.remote.log-received-messages = on
akka.logLevel = DEBUG
akka.actor.debug.autoreceive = on
akka.actor.debug.lifecycle = on
akka.actor.debug.receive = on
akka.log-config-on-start = on
akka.remote.quarantine-systems-for = off
[DEBUG] [10/30/2013 08:40:30.230] [main] [EventStream] StandardOutLogger
started
[DEBUG] [10/30/2013 08:40:30.438]
[sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/]
started (akka.actor.LocalActorRefProvider$Guardian@4bf54c5f)
[DEBUG] [10/30/2013 08:40:30.446]
[sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
started (akka.actor.LocalActorRefProvider$Guardian@72608760)
[DEBUG] [10/30/2013 08:40:30.447]
[sparkExecutor-akka.actor.default-dispatcher-4]
[akka://sparkExecutor/system] started
(akka.actor.LocalActorRefProvider$SystemGuardian@1f57ea4a)
[DEBUG] [10/30/2013 08:40:30.454]
[sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
supervising Actor[akka://sparkExecutor/user]
[DEBUG] [10/30/2013 08:40:30.454]
[sparkExecutor-akka.actor.default-dispatcher-2] [akka://sparkExecutor/] now
supervising Actor[akka://sparkExecutor/system]
[DEBUG] [10/30/2013 08:40:30.468]
[sparkExecutor-akka.actor.default-dispatcher-3] [akka://sparkExecutor/user]
now monitoring Actor[akka://sparkExecutor/system]
[DEBUG] [10/30/2013 08:40:30.468]
[sparkExecutor-akka.actor.default-dispatcher-4]
[akka://sparkExecutor/system] now monitoring Actor[akka://sparkExecutor/]
[DEBUG] [10/30/2013 08:40:30.476]
[sparkExecutor-akka.actor.default-dispatcher-3]
[akka://sparkExecutor/system/log1-Slf4jLogger] started
(akka.event.slf4j.Slf4jLogger@24988707)
[DEBUG] [10/30/2013 08:40:30.477]
[sparkExecutor-akka.actor.default-dispatcher-4]
[akka://sparkExecutor/system] now supervising
Actor[akka://sparkExecutor/system/log1-Slf4jLogger#719056881]

(followed by similar mesages for the "spark" system)

I dunno if this means much more to you, but it seems to me that for some
reason the executor decides to disconnect from the master -- unfortunately
we don't know why.  I think my logging configuration is not getting applied
correctly, or "log-sent-messages" & "log-received-messages" don't do what I
think they do ... something conflicting must be turing that logging off.
There are a zillion different remoting settings:
http://doc.akka.io/docs/akka/snapshot/scala/remoting.html

I feel like I really need to get those messages on why it disconnected to
know which ones to play with.  Any ideas for config changes to see those
messages?

thanks




On Wed, Oct 30, 2013 at 10:09 AM, Prashant Sharma <sc...@gmail.com>wrote:

> Can you apply this patch too and check the logs of Driver and worker.
>
> diff --git
> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> index b6f0ec9..ad0ebf7 100644
> ---
> a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> +++
> b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
> @@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
> ClusterScheduler, actorSystem: Actor
>      // Remove a disconnected slave from the cluster
>      def removeExecutor(executorId: String, reason: String) {
>        if (executorActor.contains(executorId)) {
> -        logInfo("Executor " + executorId + " disconnected, so removing
> it")
> +        logInfo("Executor " + executorId + " disconnected, so removing
> it, reason:" + reason)
>          val numCores = freeCores(executorId)
>          actorToExecutorId -= executorActor(executorId)
>          addressToExecutorId -= executorAddress(executorId)
>
>
>
>
> On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com>wrote:
>
>> I just realized something about the failing stages -- they generally
>> occur in steps like this:
>>
>> rdd.mapPartitions{itr =>
>>   val myCounters = initializeSomeDataStructure()
>>   itr.foreach{
>>     //update myCounter in here
>>     ...
>>   }
>>
>>   myCounters.iterator.map{
>>     //some other transformation here ...
>>   }
>> }
>>
>> that is, as a partition is processed, nothing gets output, we just
>> accumulate some values.  Only at the end of the partition do we output some
>> accumulated values.
>>
>> These stages don't always fail, and generally they do succeed after the
>> executor has died and a new one has started -- so I'm pretty confident its
>> not a problem w/ the code.  But maybe we need to add something like a
>> periodic heartbeat in this kind of operation?
>>
>>
>>
>> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> I'm gonna try turning on more akka debugging msgs as described at
>>> http://akka.io/faq/
>>> and
>>>
>>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>>
>>> unfortunately that will require a patch to spark, but hopefully that
>>> will give us more info to go on ...
>>>
>>>
>>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>
>>>> I have things running (from scala 2.10 branch) for over 3-4 hours now
>>>> without a problem and my jobs write data about the same as you suggested.
>>>> My cluster size is 7 nodes and not *congested* for memory. I going to leave
>>>> jobs running all night long. Meanwhile I had encourage you to try to spot
>>>> the problem such that it is reproducible that can help a ton in fixing the
>>>> issue.
>>>>
>>>> Thanks for testing and reporting your experience. I still feel there is
>>>> something else wrong !. About tolerance for network connection timeouts,
>>>> setting those properties should work, but I am afraid about Disassociation
>>>> Event though. I will have to check this is indeed hard to reproduce bug if
>>>> it is, I mean how do I simulate network delays ?
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>>
>>>>> At first I thought it must be some temporary network problem too --
>>>>> but the times between receiving task completion events from an executor and
>>>>> declaring it failed are really small, so I didn't think that could possibly
>>>>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>>>>> help.  Or maybe there are some other spark / akka properities we should be
>>>>> setting?  It certainly should be resilient to such a temporary network
>>>>> issue, if that is the problem.
>>>>>
>>>>> btw, I think I've noticed this happens most often during
>>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>>> total for the entire stage).
>>>>>
>>>>> thanks
>>>>>
>>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <scrapcodes@gmail.com
>>>>> > wrote:
>>>>>
>>>>>> Are you using mesos ? I admit to have not properly tested things on
>>>>>> mesos though.
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>>> scrapcodes@gmail.com> wrote:
>>>>>>
>>>>>>> Those log messages are new to the Akka 2.2 and are usually seen when
>>>>>>> a node is disassociated with other by either a network failure or even
>>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>>
>>>>>>> I had like to have more details on the long jobs though, how long ?
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>>
>>>>>>>> We've been testing out the 2.10 branch of spark, and we're running
>>>>>>>> into some issues were akka disconnects from the executors after a while.
>>>>>>>> We ran some simple tests first, and all was well, so we started upgrading
>>>>>>>> our whole codebase to 2.10.  Everything seemed to be working, but then we
>>>>>>>> noticed that when we run long jobs, and then things start failing.
>>>>>>>>
>>>>>>>>
>>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>>
>>>>>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17]
>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19]
>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Generally within a few seconds after the first such message, there
>>>>>>>> are a bunch more, and then the executor is marked as failed, and a new one
>>>>>>>> is started:
>>>>>>>>
>>>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3]
>>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered.
>>>>>>>> [10] dead letters encountered, no more dead letters will be logged. This
>>>>>>>> logging can be turned off or adjusted with configuration settings
>>>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17]
>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>>
>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17]
>>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>>> app-20131029110000-0000/2 on
>>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>>
>>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18]
>>>>>>>> ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association
>>>>>>>> failed with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>>> akka.remote.EndpointAssociationException: Association failed with
>>>>>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>>> Caused by:
>>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Looking in the logs of the failed executor, there are some similar
>>>>>>>> messages about undeliverable messages, but I don't see any reason:
>>>>>>>>
>>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>>
>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1]
>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2]
>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3]
>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>>>> terminated or disconnected! Shutting down.
>>>>>>>>
>>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4]
>>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>>
>>>>>>>>
>>>>>>>> After this happens, spark does launch a new executor successfully,
>>>>>>>> and continue the job.  Sometimes, the job just continues happily and there
>>>>>>>> aren't any other problems.  However, that executor may have to run a bunch
>>>>>>>> of steps to re-compute some cached RDDs -- and during that time, another
>>>>>>>> executor may crash similarly, and then we end up in a never ending loop, of
>>>>>>>> one executor crashing, then trying to reload data, while the others sit
>>>>>>>> around.
>>>>>>>>
>>>>>>>> I have no idea what is triggering this behavior -- there isn't any
>>>>>>>> particular point in the job that it regularly occurs at.  Certain steps
>>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>>
>>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>>
>>>>>>>> but those settings didn't seem to help the problem at all.  I
>>>>>>>> figure it must be some configuration with the new version of akka that
>>>>>>>> we're missing, but we haven't found anything.  Any ideas?
>>>>>>>>
>>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>>
>>>>>>>> thanks,
>>>>>>>> Imran
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> s
>>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> s
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> s
>>>>
>>>
>>>
>>
>
>
> --
> s
>

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Can you apply this patch too and check the logs of Driver and worker.

diff --git
a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index b6f0ec9..ad0ebf7 100644
---
a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++
b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler:
ClusterScheduler, actorSystem: Actor
     // Remove a disconnected slave from the cluster
     def removeExecutor(executorId: String, reason: String) {
       if (executorActor.contains(executorId)) {
-        logInfo("Executor " + executorId + " disconnected, so removing it")
+        logInfo("Executor " + executorId + " disconnected, so removing it,
reason:" + reason)
         val numCores = freeCores(executorId)
         actorToExecutorId -= executorActor(executorId)
         addressToExecutorId -= executorAddress(executorId)




On Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <im...@quantifind.com> wrote:

> I just realized something about the failing stages -- they generally occur
> in steps like this:
>
> rdd.mapPartitions{itr =>
>   val myCounters = initializeSomeDataStructure()
>   itr.foreach{
>     //update myCounter in here
>     ...
>   }
>
>   myCounters.iterator.map{
>     //some other transformation here ...
>   }
> }
>
> that is, as a partition is processed, nothing gets output, we just
> accumulate some values.  Only at the end of the partition do we output some
> accumulated values.
>
> These stages don't always fail, and generally they do succeed after the
> executor has died and a new one has started -- so I'm pretty confident its
> not a problem w/ the code.  But maybe we need to add something like a
> periodic heartbeat in this kind of operation?
>
>
>
> On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com>wrote:
>
>> I'm gonna try turning on more akka debugging msgs as described at
>> http://akka.io/faq/
>> and
>>
>> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>>
>> unfortunately that will require a patch to spark, but hopefully that will
>> give us more info to go on ...
>>
>>
>> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> I have things running (from scala 2.10 branch) for over 3-4 hours now
>>> without a problem and my jobs write data about the same as you suggested.
>>> My cluster size is 7 nodes and not *congested* for memory. I going to leave
>>> jobs running all night long. Meanwhile I had encourage you to try to spot
>>> the problem such that it is reproducible that can help a ton in fixing the
>>> issue.
>>>
>>> Thanks for testing and reporting your experience. I still feel there is
>>> something else wrong !. About tolerance for network connection timeouts,
>>> setting those properties should work, but I am afraid about Disassociation
>>> Event though. I will have to check this is indeed hard to reproduce bug if
>>> it is, I mean how do I simulate network delays ?
>>>
>>>
>>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>>
>>>> At first I thought it must be some temporary network problem too -- but
>>>> the times between receiving task completion events from an executor and
>>>> declaring it failed are really small, so I didn't think that could possibly
>>>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>>>> help.  Or maybe there are some other spark / akka properities we should be
>>>> setting?  It certainly should be resilient to such a temporary network
>>>> issue, if that is the problem.
>>>>
>>>> btw, I think I've noticed this happens most often during
>>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>>> total for the entire stage).
>>>>
>>>> thanks
>>>>
>>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>>
>>>>> Are you using mesos ? I admit to have not properly tested things on
>>>>> mesos though.
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <
>>>>> scrapcodes@gmail.com> wrote:
>>>>>
>>>>>> Those log messages are new to the Akka 2.2 and are usually seen when
>>>>>> a node is disassociated with other by either a network failure or even
>>>>>> clean shutdown. This suggests some network issue to me, are you running on
>>>>>> EC2 ? It might be a temporary thing in that case.
>>>>>>
>>>>>> I had like to have more details on the long jobs though, how long ?
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>>
>>>>>>> We've been testing out the 2.10 branch of spark, and we're running
>>>>>>> into some issues were akka disconnects from the executors after a while.
>>>>>>> We ran some simple tests first, and all was well, so we started upgrading
>>>>>>> our whole codebase to 2.10.  Everything seemed to be working, but then we
>>>>>>> noticed that when we run long jobs, and then things start failing.
>>>>>>>
>>>>>>>
>>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>>> undeliverable messages sent to deadLetters:
>>>>>>>
>>>>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17]
>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19]
>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Generally within a few seconds after the first such message, there
>>>>>>> are a bunch more, and then the executor is marked as failed, and a new one
>>>>>>> is started:
>>>>>>>
>>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3]
>>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>> Actor[akka://spark/deadLetters] to
>>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10]
>>>>>>> dead letters encountered, no more dead letters will be logged. This logging
>>>>>>> can be turned off or adjusted with configuration settings
>>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17]
>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>>
>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17]
>>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>>> app-20131029110000-0000/2 on
>>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>>
>>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18]
>>>>>>> ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed
>>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>>> akka.remote.EndpointAssociationException: Association failed with
>>>>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>>> Caused by:
>>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Looking in the logs of the failed executor, there are some similar
>>>>>>> messages about undeliverable messages, but I don't see any reason:
>>>>>>>
>>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>>
>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1]
>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2]
>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3]
>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>>> terminated or disconnected! Shutting down.
>>>>>>>
>>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4]
>>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>>
>>>>>>>
>>>>>>> After this happens, spark does launch a new executor successfully,
>>>>>>> and continue the job.  Sometimes, the job just continues happily and there
>>>>>>> aren't any other problems.  However, that executor may have to run a bunch
>>>>>>> of steps to re-compute some cached RDDs -- and during that time, another
>>>>>>> executor may crash similarly, and then we end up in a never ending loop, of
>>>>>>> one executor crashing, then trying to reload data, while the others sit
>>>>>>> around.
>>>>>>>
>>>>>>> I have no idea what is triggering this behavior -- there isn't any
>>>>>>> particular point in the job that it regularly occurs at.  Certain steps
>>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>>> can be actively completing stages just seconds before this failure
>>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>>
>>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>>     -Dspark.akka.frameSize=150
>>>>>>>     -Dspark.akka.timeout=120
>>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>>
>>>>>>> but those settings didn't seem to help the problem at all.  I figure
>>>>>>> it must be some configuration with the new version of akka that we're
>>>>>>> missing, but we haven't found anything.  Any ideas?
>>>>>>>
>>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>>
>>>>>>> thanks,
>>>>>>> Imran
>>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> s
>>>>>>
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> s
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
I just realized something about the failing stages -- they generally occur
in steps like this:

rdd.mapPartitions{itr =>
  val myCounters = initializeSomeDataStructure()
  itr.foreach{
    //update myCounter in here
    ...
  }

  myCounters.iterator.map{
    //some other transformation here ...
  }
}

that is, as a partition is processed, nothing gets output, we just
accumulate some values.  Only at the end of the partition do we output some
accumulated values.

These stages don't always fail, and generally they do succeed after the
executor has died and a new one has started -- so I'm pretty confident its
not a problem w/ the code.  But maybe we need to add something like a
periodic heartbeat in this kind of operation?



On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid <im...@quantifind.com> wrote:

> I'm gonna try turning on more akka debugging msgs as described at
> http://akka.io/faq/
> and
>
> http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations
>
> unfortunately that will require a patch to spark, but hopefully that will
> give us more info to go on ...
>
>
> On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> I have things running (from scala 2.10 branch) for over 3-4 hours now
>> without a problem and my jobs write data about the same as you suggested.
>> My cluster size is 7 nodes and not *congested* for memory. I going to leave
>> jobs running all night long. Meanwhile I had encourage you to try to spot
>> the problem such that it is reproducible that can help a ton in fixing the
>> issue.
>>
>> Thanks for testing and reporting your experience. I still feel there is
>> something else wrong !. About tolerance for network connection timeouts,
>> setting those properties should work, but I am afraid about Disassociation
>> Event though. I will have to check this is indeed hard to reproduce bug if
>> it is, I mean how do I simulate network delays ?
>>
>>
>> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> This is a spark-standalone setup (not mesos), on our own cluster.
>>>
>>> At first I thought it must be some temporary network problem too -- but
>>> the times between receiving task completion events from an executor and
>>> declaring it failed are really small, so I didn't think that could possibly
>>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>>> help.  Or maybe there are some other spark / akka properities we should be
>>> setting?  It certainly should be resilient to such a temporary network
>>> issue, if that is the problem.
>>>
>>> btw, I think I've noticed this happens most often during
>>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>>> total for the entire stage).
>>>
>>> thanks
>>>
>>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>
>>>> Are you using mesos ? I admit to have not properly tested things on
>>>> mesos though.
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <scrapcodes@gmail.com
>>>> > wrote:
>>>>
>>>>> Those log messages are new to the Akka 2.2 and are usually seen when a
>>>>> node is disassociated with other by either a network failure or even clean
>>>>> shutdown. This suggests some network issue to me, are you running on EC2 ?
>>>>> It might be a temporary thing in that case.
>>>>>
>>>>> I had like to have more details on the long jobs though, how long ?
>>>>>
>>>>>
>>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>>
>>>>>> We've been testing out the 2.10 branch of spark, and we're running
>>>>>> into some issues were akka disconnects from the executors after a while.
>>>>>> We ran some simple tests first, and all was well, so we started upgrading
>>>>>> our whole codebase to 2.10.  Everything seemed to be working, but then we
>>>>>> noticed that when we run long jobs, and then things start failing.
>>>>>>
>>>>>>
>>>>>> The first suspicious thing is that we get akka warnings about
>>>>>> undeliverable messages sent to deadLetters:
>>>>>>
>>>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17]
>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>> Actor[akka://spark/deadLetters] to
>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19]
>>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>> Actor[akka://spark/deadLetters] to
>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>>
>>>>>>
>>>>>> Generally within a few seconds after the first such message, there
>>>>>> are a bunch more, and then the executor is marked as failed, and a new one
>>>>>> is started:
>>>>>>
>>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
>>>>>> akka.actor.LocalActorRef - Message
>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>> Actor[akka://spark/deadLetters] to
>>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10]
>>>>>> dead letters encountered, no more dead letters will be logged. This logging
>>>>>> can be turned off or adjusted with configuration settings
>>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17]
>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>>
>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17]
>>>>>> INFO  org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>>> app-20131029110000-0000/2 on
>>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>>
>>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18]
>>>>>> ERROR akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed
>>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>>> akka.remote.EndpointAssociationException: Association failed with
>>>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>>> Caused by:
>>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>>
>>>>>>
>>>>>>
>>>>>> Looking in the logs of the failed executor, there are some similar
>>>>>> messages about undeliverable messages, but I don't see any reason:
>>>>>>
>>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>>
>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1]
>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2]
>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3]
>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>>>>> terminated or disconnected! Shutting down.
>>>>>>
>>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4]
>>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>>> configuration settings 'akka.log-dead-letters' and
>>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>>
>>>>>>
>>>>>> After this happens, spark does launch a new executor successfully,
>>>>>> and continue the job.  Sometimes, the job just continues happily and there
>>>>>> aren't any other problems.  However, that executor may have to run a bunch
>>>>>> of steps to re-compute some cached RDDs -- and during that time, another
>>>>>> executor may crash similarly, and then we end up in a never ending loop, of
>>>>>> one executor crashing, then trying to reload data, while the others sit
>>>>>> around.
>>>>>>
>>>>>> I have no idea what is triggering this behavior -- there isn't any
>>>>>> particular point in the job that it regularly occurs at.  Certain steps
>>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>>> can be actively completing stages just seconds before this failure
>>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>>
>>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>>     -Dspark.akka.frameSize=150
>>>>>>     -Dspark.akka.timeout=120
>>>>>>     -Dspark.akka.askTimeout=30
>>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>>
>>>>>> but those settings didn't seem to help the problem at all.  I figure
>>>>>> it must be some configuration with the new version of akka that we're
>>>>>> missing, but we haven't found anything.  Any ideas?
>>>>>>
>>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The
>>>>>> failures occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>>
>>>>>> thanks,
>>>>>> Imran
>>>>>>
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> s
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> s
>>>>
>>>
>>>
>>
>>
>> --
>> s
>>
>
>

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
I'm gonna try turning on more akka debugging msgs as described at
http://akka.io/faq/
and
http://doc.akka.io/docs/akka/current/scala/testing.html#Tracing_Actor_Invocations

unfortunately that will require a patch to spark, but hopefully that will
give us more info to go on ...


On Wed, Oct 30, 2013 at 8:10 AM, Prashant Sharma <sc...@gmail.com>wrote:

> I have things running (from scala 2.10 branch) for over 3-4 hours now
> without a problem and my jobs write data about the same as you suggested.
> My cluster size is 7 nodes and not *congested* for memory. I going to leave
> jobs running all night long. Meanwhile I had encourage you to try to spot
> the problem such that it is reproducible that can help a ton in fixing the
> issue.
>
> Thanks for testing and reporting your experience. I still feel there is
> something else wrong !. About tolerance for network connection timeouts,
> setting those properties should work, but I am afraid about Disassociation
> Event though. I will have to check this is indeed hard to reproduce bug if
> it is, I mean how do I simulate network delays ?
>
>
> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com>wrote:
>
>> This is a spark-standalone setup (not mesos), on our own cluster.
>>
>> At first I thought it must be some temporary network problem too -- but
>> the times between receiving task completion events from an executor and
>> declaring it failed are really small, so I didn't think that could possibly
>> be it.  Plus we tried increasing various akka timeouts, but that didn't
>> help.  Or maybe there are some other spark / akka properities we should be
>> setting?  It certainly should be resilient to such a temporary network
>> issue, if that is the problem.
>>
>> btw, I think I've noticed this happens most often during
>> ShuffleMapTasks.  The tasks write out very small amounts of data (64 MB
>> total for the entire stage).
>>
>> thanks
>>
>> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> Are you using mesos ? I admit to have not properly tested things on
>>> mesos though.
>>>
>>>
>>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>>
>>>> Those log messages are new to the Akka 2.2 and are usually seen when a
>>>> node is disassociated with other by either a network failure or even clean
>>>> shutdown. This suggests some network issue to me, are you running on EC2 ?
>>>> It might be a temporary thing in that case.
>>>>
>>>> I had like to have more details on the long jobs though, how long ?
>>>>
>>>>
>>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>>
>>>>> We've been testing out the 2.10 branch of spark, and we're running
>>>>> into some issues were akka disconnects from the executors after a while.
>>>>> We ran some simple tests first, and all was well, so we started upgrading
>>>>> our whole codebase to 2.10.  Everything seemed to be working, but then we
>>>>> noticed that when we run long jobs, and then things start failing.
>>>>>
>>>>>
>>>>> The first suspicious thing is that we get akka warnings about
>>>>> undeliverable messages sent to deadLetters:
>>>>>
>>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17]
>>>>> INFO  akka.actor.LocalActorRef - Message
>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>> Actor[akka://spark/deadLetters] to
>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO
>>>>> akka.actor.LocalActorRef - Message
>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>> Actor[akka://spark/deadLetters] to
>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>>
>>>>>
>>>>> Generally within a few seconds after the first such message, there are
>>>>> a bunch more, and then the executor is marked as failed, and a new one is
>>>>> started:
>>>>>
>>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
>>>>> akka.actor.LocalActorRef - Message
>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>> Actor[akka://spark/deadLetters] to
>>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10]
>>>>> dead letters encountered, no more dead letters will be logged. This logging
>>>>> can be turned off or adjusted with configuration settings
>>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO
>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>>
>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO
>>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>>> app-20131029110000-0000/2 on
>>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>>
>>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR
>>>>> akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed
>>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>>>>> akka.remote.EndpointAssociationException: Association failed with
>>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>>> Caused by:
>>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>>
>>>>>
>>>>>
>>>>> Looking in the logs of the failed executor, there are some similar
>>>>> messages about undeliverable messages, but I don't see any reason:
>>>>>
>>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>>
>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1]
>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>> configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2]
>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>> configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3]
>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>> configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>>>> terminated or disconnected! Shutting down.
>>>>>
>>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4]
>>>>> dead letters encountered. This logging can be turned off or adjusted with
>>>>> configuration settings 'akka.log-dead-letters' and
>>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>>
>>>>>
>>>>> After this happens, spark does launch a new executor successfully, and
>>>>> continue the job.  Sometimes, the job just continues happily and there
>>>>> aren't any other problems.  However, that executor may have to run a bunch
>>>>> of steps to re-compute some cached RDDs -- and during that time, another
>>>>> executor may crash similarly, and then we end up in a never ending loop, of
>>>>> one executor crashing, then trying to reload data, while the others sit
>>>>> around.
>>>>>
>>>>> I have no idea what is triggering this behavior -- there isn't any
>>>>> particular point in the job that it regularly occurs at.  Certain steps
>>>>> seem more prone to this, but there isn't any step which regularly causes
>>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>>> can be actively completing stages just seconds before this failure
>>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>>
>>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>>     -Dspark.akka.frameSize=150
>>>>>     -Dspark.akka.timeout=120
>>>>>     -Dspark.akka.askTimeout=30
>>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>>
>>>>> but those settings didn't seem to help the problem at all.  I figure
>>>>> it must be some configuration with the new version of akka that we're
>>>>> missing, but we haven't found anything.  Any ideas?
>>>>>
>>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures
>>>>> occur on the tip of the scala-2.10 branch (5429d62d)
>>>>>
>>>>> thanks,
>>>>> Imran
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> s
>>>>
>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>
>
> --
> s
>

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
I have things running (from scala 2.10 branch) for over 3-4 hours now
without a problem and my jobs write data about the same as you suggested.
My cluster size is 7 nodes and not *congested* for memory. I going to leave
jobs running all night long. Meanwhile I had encourage you to try to spot
the problem such that it is reproducible that can help a ton in fixing the
issue.

Thanks for testing and reporting your experience. I still feel there is
something else wrong !. About tolerance for network connection timeouts,
setting those properties should work, but I am afraid about Disassociation
Event though. I will have to check this is indeed hard to reproduce bug if
it is, I mean how do I simulate network delays ?


On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid <im...@quantifind.com> wrote:

> This is a spark-standalone setup (not mesos), on our own cluster.
>
> At first I thought it must be some temporary network problem too -- but
> the times between receiving task completion events from an executor and
> declaring it failed are really small, so I didn't think that could possibly
> be it.  Plus we tried increasing various akka timeouts, but that didn't
> help.  Or maybe there are some other spark / akka properities we should be
> setting?  It certainly should be resilient to such a temporary network
> issue, if that is the problem.
>
> btw, I think I've noticed this happens most often during ShuffleMapTasks.
> The tasks write out very small amounts of data (64 MB total for the entire
> stage).
>
> thanks
>
> On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> Are you using mesos ? I admit to have not properly tested things on mesos
>> though.
>>
>>
>> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <sc...@gmail.com>wrote:
>>
>>> Those log messages are new to the Akka 2.2 and are usually seen when a
>>> node is disassociated with other by either a network failure or even clean
>>> shutdown. This suggests some network issue to me, are you running on EC2 ?
>>> It might be a temporary thing in that case.
>>>
>>> I had like to have more details on the long jobs though, how long ?
>>>
>>>
>>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>>>
>>>> We've been testing out the 2.10 branch of spark, and we're running into
>>>> some issues were akka disconnects from the executors after a while.  We ran
>>>> some simple tests first, and all was well, so we started upgrading our
>>>> whole codebase to 2.10.  Everything seemed to be working, but then we
>>>> noticed that when we run long jobs, and then things start failing.
>>>>
>>>>
>>>> The first suspicious thing is that we get akka warnings about
>>>> undeliverable messages sent to deadLetters:
>>>>
>>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO
>>>> akka.actor.LocalActorRef - Message
>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>> Actor[akka://spark/deadLetters] to
>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO
>>>> akka.actor.LocalActorRef - Message
>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>> Actor[akka://spark/deadLetters] to
>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>>
>>>>
>>>> Generally within a few seconds after the first such message, there are
>>>> a bunch more, and then the executor is marked as failed, and a new one is
>>>> started:
>>>>
>>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
>>>> akka.actor.LocalActorRef - Message
>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>> Actor[akka://spark/deadLetters] to
>>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10]
>>>> dead letters encountered, no more dead letters will be logged. This logging
>>>> can be turned off or adjusted with configuration settings
>>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO
>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>>
>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO
>>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>>> app-20131029110000-0000/2 on
>>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>>> dhd2.quantifind.com:51544) with 24 cores
>>>>
>>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR
>>>> akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed
>>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>>>> akka.remote.EndpointAssociationException: Association failed with
>>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>>> Caused by:
>>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>>
>>>>
>>>>
>>>> Looking in the logs of the failed executor, there are some similar
>>>> messages about undeliverable messages, but I don't see any reason:
>>>>
>>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>>
>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead
>>>> letters encountered. This logging can be turned off or adjusted with
>>>> configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead
>>>> letters encountered. This logging can be turned off or adjusted with
>>>> configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead
>>>> letters encountered. This logging can be turned off or adjusted with
>>>> configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>>> terminated or disconnected! Shutting down.
>>>>
>>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>>> Actor[akka://sparkExecutor/deadLetters] to
>>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead
>>>> letters encountered. This logging can be turned off or adjusted with
>>>> configuration settings 'akka.log-dead-letters' and
>>>> 'akka.log-dead-letters-during-shutdown'.
>>>>
>>>>
>>>> After this happens, spark does launch a new executor successfully, and
>>>> continue the job.  Sometimes, the job just continues happily and there
>>>> aren't any other problems.  However, that executor may have to run a bunch
>>>> of steps to re-compute some cached RDDs -- and during that time, another
>>>> executor may crash similarly, and then we end up in a never ending loop, of
>>>> one executor crashing, then trying to reload data, while the others sit
>>>> around.
>>>>
>>>> I have no idea what is triggering this behavior -- there isn't any
>>>> particular point in the job that it regularly occurs at.  Certain steps
>>>> seem more prone to this, but there isn't any step which regularly causes
>>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>>> can be actively completing stages just seconds before this failure
>>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>>
>>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>>     -Dspark.akka.frameSize=150
>>>>     -Dspark.akka.timeout=120
>>>>     -Dspark.akka.askTimeout=30
>>>>     -Dspark.akka.logLifecycleEvents=true
>>>>
>>>> but those settings didn't seem to help the problem at all.  I figure it
>>>> must be some configuration with the new version of akka that we're missing,
>>>> but we haven't found anything.  Any ideas?
>>>>
>>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures
>>>> occur on the tip of the scala-2.10 branch (5429d62d)
>>>>
>>>> thanks,
>>>> Imran
>>>>
>>>
>>>
>>>
>>> --
>>> s
>>>
>>
>>
>>
>> --
>> s
>>
>
>


-- 
s

Re: executor failures w/ scala 2.10

Posted by Imran Rashid <im...@quantifind.com>.
This is a spark-standalone setup (not mesos), on our own cluster.

At first I thought it must be some temporary network problem too -- but the
times between receiving task completion events from an executor and
declaring it failed are really small, so I didn't think that could possibly
be it.  Plus we tried increasing various akka timeouts, but that didn't
help.  Or maybe there are some other spark / akka properities we should be
setting?  It certainly should be resilient to such a temporary network
issue, if that is the problem.

btw, I think I've noticed this happens most often during ShuffleMapTasks.
The tasks write out very small amounts of data (64 MB total for the entire
stage).

thanks

On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <sc...@gmail.com>wrote:

> Are you using mesos ? I admit to have not properly tested things on mesos
> though.
>
>
> On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <sc...@gmail.com>wrote:
>
>> Those log messages are new to the Akka 2.2 and are usually seen when a
>> node is disassociated with other by either a network failure or even clean
>> shutdown. This suggests some network issue to me, are you running on EC2 ?
>> It might be a temporary thing in that case.
>>
>> I had like to have more details on the long jobs though, how long ?
>>
>>
>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>>
>>> We've been testing out the 2.10 branch of spark, and we're running into
>>> some issues were akka disconnects from the executors after a while.  We ran
>>> some simple tests first, and all was well, so we started upgrading our
>>> whole codebase to 2.10.  Everything seemed to be working, but then we
>>> noticed that when we run long jobs, and then things start failing.
>>>
>>>
>>> The first suspicious thing is that we get akka warnings about
>>> undeliverable messages sent to deadLetters:
>>>
>>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO
>>> akka.actor.LocalActorRef - Message
>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>> Actor[akka://spark/deadLetters] to
>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>> was not delivered. [4] dead letters encountered. This logging can be turned
>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO
>>> akka.actor.LocalActorRef - Message
>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>> Actor[akka://spark/deadLetters] to
>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>>> was not delivered. [5] dead letters encountered. This logging can be turned
>>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>>
>>>
>>> Generally within a few seconds after the first such message, there are a
>>> bunch more, and then the executor is marked as failed, and a new one is
>>> started:
>>>
>>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
>>> akka.actor.LocalActorRef - Message
>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>> Actor[akka://spark/deadLetters] to
>>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10]
>>> dead letters encountered, no more dead letters will be logged. This logging
>>> can be turned off or adjusted with configuration settings
>>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>>
>>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO
>>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>>
>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO
>>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>>> app-20131029110000-0000/2 on
>>> worker-20131029105824-dhd2.quantifind.com-51544 (
>>> dhd2.quantifind.com:51544) with 24 cores
>>>
>>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR
>>> akka.remote.EndpointWriter - AssociationError [akka.tcp://
>>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed
>>> with [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>>> akka.remote.EndpointAssociationException: Association failed with
>>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>>> Caused by:
>>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>>
>>>
>>>
>>> Looking in the logs of the failed executor, there are some similar
>>> messages about undeliverable messages, but I don't see any reason:
>>>
>>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>>
>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead
>>> letters encountered. This logging can be turned off or adjusted with
>>> configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>> Actor[akka://sparkExecutor/deadLetters] to
>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead
>>> letters encountered. This logging can be turned off or adjusted with
>>> configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>> [akka.remote.transport.AssociationHandle$Disassociated] from
>>> Actor[akka://sparkExecutor/deadLetters] to
>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead
>>> letters encountered. This logging can be turned off or adjusted with
>>> configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>>> terminated or disconnected! Shutting down.
>>>
>>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>>> Actor[akka://sparkExecutor/deadLetters] to
>>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead
>>> letters encountered. This logging can be turned off or adjusted with
>>> configuration settings 'akka.log-dead-letters' and
>>> 'akka.log-dead-letters-during-shutdown'.
>>>
>>>
>>> After this happens, spark does launch a new executor successfully, and
>>> continue the job.  Sometimes, the job just continues happily and there
>>> aren't any other problems.  However, that executor may have to run a bunch
>>> of steps to re-compute some cached RDDs -- and during that time, another
>>> executor may crash similarly, and then we end up in a never ending loop, of
>>> one executor crashing, then trying to reload data, while the others sit
>>> around.
>>>
>>> I have no idea what is triggering this behavior -- there isn't any
>>> particular point in the job that it regularly occurs at.  Certain steps
>>> seem more prone to this, but there isn't any step which regularly causes
>>> the problem.  In a long pipeline of steps, though, that loop becomes very
>>> likely.  I don't think its a timeout issue -- the initial failing executors
>>> can be actively completing stages just seconds before this failure
>>> happens.  We did try adjusting some of the spark / akka timeouts:
>>>
>>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>>     -Dspark.akka.frameSize=150
>>>     -Dspark.akka.timeout=120
>>>     -Dspark.akka.askTimeout=30
>>>     -Dspark.akka.logLifecycleEvents=true
>>>
>>> but those settings didn't seem to help the problem at all.  I figure it
>>> must be some configuration with the new version of akka that we're missing,
>>> but we haven't found anything.  Any ideas?
>>>
>>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures
>>> occur on the tip of the scala-2.10 branch (5429d62d)
>>>
>>> thanks,
>>> Imran
>>>
>>
>>
>>
>> --
>> s
>>
>
>
>
> --
> s
>

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Are you using mesos ? I admit to have not properly tested things on mesos
though.


On Wed, Oct 30, 2013 at 11:31 AM, Prashant Sharma <sc...@gmail.com>wrote:

> Those log messages are new to the Akka 2.2 and are usually seen when a
> node is disassociated with other by either a network failure or even clean
> shutdown. This suggests some network issue to me, are you running on EC2 ?
> It might be a temporary thing in that case.
>
> I had like to have more details on the long jobs though, how long ?
>
>
> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com>wrote:
>
>> We've been testing out the 2.10 branch of spark, and we're running into
>> some issues were akka disconnects from the executors after a while.  We ran
>> some simple tests first, and all was well, so we started upgrading our
>> whole codebase to 2.10.  Everything seemed to be working, but then we
>> noticed that when we run long jobs, and then things start failing.
>>
>>
>> The first suspicious thing is that we get akka warnings about
>> undeliverable messages sent to deadLetters:
>>
>> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO
>> akka.actor.LocalActorRef - Message
>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>> Actor[akka://spark/deadLetters] to
>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>> was not delivered. [4] dead letters encountered. This logging can be turned
>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO
>> akka.actor.LocalActorRef - Message
>> [akka.remote.transport.AssociationHandle$Disassociated] from
>> Actor[akka://spark/deadLetters] to
>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
>> was not delivered. [5] dead letters encountered. This logging can be turned
>> off or adjusted with configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>>
>>
>> Generally within a few seconds after the first such message, there are a
>> bunch more, and then the executor is marked as failed, and a new one is
>> started:
>>
>> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
>> akka.actor.LocalActorRef - Message
>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>> Actor[akka://spark/deadLetters] to
>> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
>> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10] dead
>> letters encountered, no more dead letters will be logged. This logging can
>> be turned off or adjusted with configuration settings
>> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>>
>> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO
>> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
>> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>>
>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO
>> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
>> app-20131029110000-0000/2 on
>> worker-20131029105824-dhd2.quantifind.com-51544 (
>> dhd2.quantifind.com:51544) with 24 cores
>>
>> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR
>> akka.remote.EndpointWriter - AssociationError [akka.tcp://
>> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
>> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed with
>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
>> akka.remote.EndpointAssociationException: Association failed with
>> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
>> Caused by:
>> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
>> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>>
>>
>>
>> Looking in the logs of the failed executor, there are some similar
>> messages about undeliverable messages, but I don't see any reason:
>>
>> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>>
>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>> [akka.actor.FSM$Timer] from Actor[akka://sparkExecutor/deadLetters] to
>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead
>> letters encountered. This logging can be turned off or adjusted with
>> configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>> [akka.remote.transport.AssociationHandle$Disassociated] from
>> Actor[akka://sparkExecutor/deadLetters] to
>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead
>> letters encountered. This logging can be turned off or adjusted with
>> configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>> [akka.remote.transport.AssociationHandle$Disassociated] from
>> Actor[akka://sparkExecutor/deadLetters] to
>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead
>> letters encountered. This logging can be turned off or adjusted with
>> configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
>> terminated or disconnected! Shutting down.
>>
>> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
>> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
>> Actor[akka://sparkExecutor/deadLetters] to
>> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
>> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead
>> letters encountered. This logging can be turned off or adjusted with
>> configuration settings 'akka.log-dead-letters' and
>> 'akka.log-dead-letters-during-shutdown'.
>>
>>
>> After this happens, spark does launch a new executor successfully, and
>> continue the job.  Sometimes, the job just continues happily and there
>> aren't any other problems.  However, that executor may have to run a bunch
>> of steps to re-compute some cached RDDs -- and during that time, another
>> executor may crash similarly, and then we end up in a never ending loop, of
>> one executor crashing, then trying to reload data, while the others sit
>> around.
>>
>> I have no idea what is triggering this behavior -- there isn't any
>> particular point in the job that it regularly occurs at.  Certain steps
>> seem more prone to this, but there isn't any step which regularly causes
>> the problem.  In a long pipeline of steps, though, that loop becomes very
>> likely.  I don't think its a timeout issue -- the initial failing executors
>> can be actively completing stages just seconds before this failure
>> happens.  We did try adjusting some of the spark / akka timeouts:
>>
>>     -Dspark.storage.blockManagerHeartBeatMs=300000
>>     -Dspark.akka.frameSize=150
>>     -Dspark.akka.timeout=120
>>     -Dspark.akka.askTimeout=30
>>     -Dspark.akka.logLifecycleEvents=true
>>
>> but those settings didn't seem to help the problem at all.  I figure it
>> must be some configuration with the new version of akka that we're missing,
>> but we haven't found anything.  Any ideas?
>>
>> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures
>> occur on the tip of the scala-2.10 branch (5429d62d)
>>
>> thanks,
>> Imran
>>
>
>
>
> --
> s
>



-- 
s

Re: executor failures w/ scala 2.10

Posted by Prashant Sharma <sc...@gmail.com>.
Those log messages are new to the Akka 2.2 and are usually seen when a node
is disassociated with other by either a network failure or even clean
shutdown. This suggests some network issue to me, are you running on EC2 ?
It might be a temporary thing in that case.

I had like to have more details on the long jobs though, how long ?


On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <im...@quantifind.com> wrote:

> We've been testing out the 2.10 branch of spark, and we're running into
> some issues were akka disconnects from the executors after a while.  We ran
> some simple tests first, and all was well, so we started upgrading our
> whole codebase to 2.10.  Everything seemed to be working, but then we
> noticed that when we run long jobs, and then things start failing.
>
>
> The first suspicious thing is that we get akka warnings about
> undeliverable messages sent to deadLetters:
>
> 22013-10-29 11:03:54,577 [spark-akka.actor.default-dispatcher-17] INFO
> akka.actor.LocalActorRef - Message
> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
> Actor[akka://spark/deadLetters] to
> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
> was not delivered. [4] dead letters encountered. This logging can be turned
> off or adjusted with configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
> 2013-10-29 11:03:54,579 [spark-akka.actor.default-dispatcher-19] INFO
> akka.actor.LocalActorRef - Message
> [akka.remote.transport.AssociationHandle$Disassociated] from
> Actor[akka://spark/deadLetters] to
> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%4010.10.5.81%3A46572-3#656094700]
> was not delivered. [5] dead letters encountered. This logging can be turned
> off or adjusted with configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
>
>
> Generally within a few seconds after the first such message, there are a
> bunch more, and then the executor is marked as failed, and a new one is
> started:
>
> 2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO
> akka.actor.LocalActorRef - Message
> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
> Actor[akka://spark/deadLetters] to
> Actor[akka://spark/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkExecutor%
> 40dhd2.quantifind.com%3A45794-6#-890135716] was not delivered. [10] dead
> letters encountered, no more dead letters will be logged. This logging can
> be turned off or adjusted with configuration settings
> 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'.
>
> 2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO
> org.apache.spark.deploy.client.Client$ClientActor - Executor updated:
> app-20131029110000-0000/1 is now FAILED (Command exited with code 1)
>
> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-17] INFO
> org.apache.spark.deploy.client.Client$ClientActor - Executor added:
> app-20131029110000-0000/2 on
> worker-20131029105824-dhd2.quantifind.com-51544 (dhd2.quantifind.com:51544)
> with 24 cores
>
> 2013-10-29 11:03:58,784 [spark-akka.actor.default-dispatcher-18] ERROR
> akka.remote.EndpointWriter - AssociationError [akka.tcp://
> spark@ddd0.quantifind.com:43068] -> [akka.tcp://
> sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed with
> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]] [
> akka.remote.EndpointAssociationException: Association failed with
> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]
> Caused by:
> akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2:
> Connection refused: dhd2.quantifind.com/10.10.5.64:45794]
>
>
>
> Looking in the logs of the failed executor, there are some similar
> messages about undeliverable messages, but I don't see any reason:
>
> 13/10/29 11:03:52 INFO executor.Executor: Finished task ID 943
>
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.actor.FSM$Timer]
> from Actor[akka://sparkExecutor/deadLetters] to
> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [1] dead
> letters encountered. This logging can be turned off or adjusted with
> configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
> [akka.remote.transport.AssociationHandle$Disassociated] from
> Actor[akka://sparkExecutor/deadLetters] to
> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [2] dead
> letters encountered. This logging can be turned off or adjusted with
> configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
> [akka.remote.transport.AssociationHandle$Disassociated] from
> Actor[akka://sparkExecutor/deadLetters] to
> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [3] dead
> letters encountered. This logging can be turned off or adjusted with
> configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
> 13/10/29 11:03:53 ERROR executor.StandaloneExecutorBackend: Driver
> terminated or disconnected! Shutting down.
>
> 13/10/29 11:03:53 INFO actor.LocalActorRef: Message
> [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from
> Actor[akka://sparkExecutor/deadLetters] to
> Actor[akka://sparkExecutor/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%
> 40ddd0.quantifind.com%3A43068-1#772172548] was not delivered. [4] dead
> letters encountered. This logging can be turned off or adjusted with
> configuration settings 'akka.log-dead-letters' and
> 'akka.log-dead-letters-during-shutdown'.
>
>
> After this happens, spark does launch a new executor successfully, and
> continue the job.  Sometimes, the job just continues happily and there
> aren't any other problems.  However, that executor may have to run a bunch
> of steps to re-compute some cached RDDs -- and during that time, another
> executor may crash similarly, and then we end up in a never ending loop, of
> one executor crashing, then trying to reload data, while the others sit
> around.
>
> I have no idea what is triggering this behavior -- there isn't any
> particular point in the job that it regularly occurs at.  Certain steps
> seem more prone to this, but there isn't any step which regularly causes
> the problem.  In a long pipeline of steps, though, that loop becomes very
> likely.  I don't think its a timeout issue -- the initial failing executors
> can be actively completing stages just seconds before this failure
> happens.  We did try adjusting some of the spark / akka timeouts:
>
>     -Dspark.storage.blockManagerHeartBeatMs=300000
>     -Dspark.akka.frameSize=150
>     -Dspark.akka.timeout=120
>     -Dspark.akka.askTimeout=30
>     -Dspark.akka.logLifecycleEvents=true
>
> but those settings didn't seem to help the problem at all.  I figure it
> must be some configuration with the new version of akka that we're missing,
> but we haven't found anything.  Any ideas?
>
> our code works fine w/ the 0.8.0 release on scala 2.9.3.  The failures
> occur on the tip of the scala-2.10 branch (5429d62d)
>
> thanks,
> Imran
>



-- 
s