Return-Path: X-Original-To: apmail-spark-user-archive@minotaur.apache.org Delivered-To: apmail-spark-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 94CC110499 for ; Wed, 30 Oct 2013 15:10:27 +0000 (UTC) Received: (qmail 47319 invoked by uid 500); 30 Oct 2013 15:10:26 -0000 Delivered-To: apmail-spark-user-archive@spark.apache.org Received: (qmail 46952 invoked by uid 500); 30 Oct 2013 15:10:20 -0000 Mailing-List: contact user-help@spark.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@spark.incubator.apache.org Delivered-To: mailing list user@spark.incubator.apache.org Received: (qmail 46942 invoked by uid 99); 30 Oct 2013 15:10:19 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Oct 2013 15:10:18 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS,WEIRD_PORT X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of scrapcodes@gmail.com designates 74.125.82.53 as permitted sender) Received: from [74.125.82.53] (HELO mail-wg0-f53.google.com) (74.125.82.53) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Oct 2013 15:10:14 +0000 Received: by mail-wg0-f53.google.com with SMTP id y10so1462493wgg.20 for ; Wed, 30 Oct 2013 08:09:53 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:from:date:message-id:subject:to :content-type; bh=bniqauVUutFRdsYfwxvspiEdTBoSL+4+iUbZg3jufWQ=; b=VS+E28C88vttfPNxd6peok2QEUXCwW9VmquXNLtuhF9taZQvNY3T3DIu2YD7QgFs8/ eroaacH+VLSSqJ0RnvCl1YnMRl1SuYpxmsg7d0UcAayuXPjDgazRyRgVoa3z2/YyEKFV lbVRT+lVfSmwrCEZrVk1BhKu743XgCMzNaIpPuOJN0cBxjN8H9OiKoJdX4SYeer66UOH Oa2JNGsH3+aw15vDePdXy7f0nxn4CdNbfgxGdR/QaoH7yJaNwEf/DXYs4F6IFylEPstn c/8vEP+jMy5Oj7jRs2VpbiBPM5i7bZA3CHtYLnUeNz91c2Z6JfXnxgvmx+m1GsGJWw7j +Yqw== X-Received: by 10.180.85.39 with SMTP id e7mr6459wiz.7.1383145793488; Wed, 30 Oct 2013 08:09:53 -0700 (PDT) MIME-Version: 1.0 Received: by 10.216.64.132 with HTTP; Wed, 30 Oct 2013 08:09:33 -0700 (PDT) In-Reply-To: References: From: Prashant Sharma Date: Wed, 30 Oct 2013 20:39:33 +0530 Message-ID: Subject: Re: executor failures w/ scala 2.10 To: user@spark.incubator.apache.org Content-Type: multipart/alternative; boundary=f46d04182562b18bff04e9f6b943 X-Virus-Checked: Checked by ClamAV on apache.org --f46d04182562b18bff04e9f6b943 Content-Type: text/plain; charset=ISO-8859-1 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 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 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 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 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 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 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 --f46d04182562b18bff04e9f6b943 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable
Can you apply this patch too and check the logs of Driver = and worker.

diff --git a/core/src/main/scala/org/ap= ache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/ma= in/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scal= a
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/StandaloneSched= ulerBackend.scala
@@ -132,7 +132,7 @@ class StandaloneSchedulerBackend(scheduler: Cluste= rScheduler, actorSystem: Actor
=A0 =A0 =A0// Remove a disconnecte= d slave from the cluster
=A0 =A0 =A0def removeExecutor(executorId= : String, reason: String) {
=A0 =A0 =A0 =A0if (executorActor.contains(executorId)) {
- = =A0 =A0 =A0 =A0logInfo("Executor " + executorId + " disconne= cted, so removing it")
+ =A0 =A0 =A0 =A0logInfo("Execut= or " + executorId + " disconnected, so removing it, reason:"= + reason)
=A0 =A0 =A0 =A0 =A0val numCores =3D freeCores(executorId)
= =A0 =A0 =A0 =A0 =A0actorToExecutorId -=3D executorActor(executorId)
=A0 =A0 =A0 =A0 =A0addressToExecutorId -=3D executorAddress(executorId)<= /div>




O= n Wed, Oct 30, 2013 at 8:18 PM, Imran Rashid <imran@quantifind.com= > wrote:
I just realized something about the failing stages -- they generally occ= ur in steps like this:

rdd.mapPartitions{itr =3D>
=A0 val myCounters =3D ini= tializeSomeDataStructure()
=A0 itr.foreach{
=A0=A0=A0 //update myCounter in here
=A0=A0=A0 ...
=A0 }
=A0
=A0 myCounters.iterator.map{
= =A0=A0=A0 //some other transformation here ...
=A0 }
}

t= hat is, as a partition is processed, nothing gets output, we just accumulat= e some values.=A0 Only at the end of the partition do we output some accumu= lated 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.=A0 But maybe we need to add somet= hing like a periodic heartbeat in this kind of operation?



On Wed, Oct 30, = 2013 at 8:56 AM, Imran Rashid <imran@quantifind.com> wrot= e:
I'm gonna try= turning on more akka debugging msgs as described at
http://akka.io/faq/
and
http://doc.akka.io/docs/a= kka/current/scala/testing.html#Tracing_Actor_Invocations

unfortunately that will require a patch to spark, but hopefully t= hat 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 wr= ite data about the same as you suggested. My cluster size is 7 nodes and no= t *congested* for memory. I going to leave jobs running all night long. Mea= nwhile I had encourage you to try to spot the problem such that it is repro= ducible that can help a ton in fixing the issue.=A0

Thanks for testing and reporting your experience. I still feel there is= something else wrong !. About tolerance for network connection timeouts, s= etting those properties should work, but I am afraid about Disassociation E= vent though. I will have to check this is indeed hard to reproduce bug if i= t 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-standa= lone setup (not mesos), on our own cluster.

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

btw, I think I've noticed this happens most often during ShuffleMap= Tasks.=A0 The tasks write out very small amounts of data (64 MB total for t= he entire stage).

thanks

On Wed, Oct 30, 2013 at 6:47 AM, Prashant Sharma <scrapcodes@g= mail.com> wrote:
Are you using mesos ? I adm= it to have not properly tested things on mesos though.=A0


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 othe= r by either a network failure or even clean shutdown. This suggests some ne= twork issue to me, are you running on EC2 ? It might be a temporary thing i= n that case.=A0

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


On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid <imra= n@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 executor= s after a while.=A0 We ran some simple tests first, and all was well, so we started upgrading our whole codebase to 2.10.=A0 Everything seemed to be= =20 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 undel= iverable messages sent to deadLetters:

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

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



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

2013-10-29 11:03:58,775 [spark-akka.actor.default-dispatcher-3] INFO=A0 akk= a.actor.LocalActorRef - Message [akka.remote.transport.ActorTransportAdapte= r$DisassociateUnderlying] from Actor[akka://spark/deadLetters] to Actor[akk= a://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=20 will be logged. This logging can be turned off or adjusted with=20 configuration settings 'akka.log-dead-letters' and=20 'akka.log-dead-letters-during-shutdown'.

2013-10-29 11:03:58,778 [spark-akka.actor.default-dispatcher-17] INFO= =A0 org.apache.spark.deploy.client.Client$ClientActor - Executor updated: a= pp-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=A0 or= g.apache.spark.deploy.client.Client$ClientActor - Executor added: app-20131= 029110000-0000/2 on worker-20131029105824-dhd2.quantifind.com-51544 (dhd2.quantifind.co= m: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.c= om:43068] -> [akka.tcp://sparkExecutor@dhd2.quantifind.com:45794]: Error [Association failed with [akka.tcp://sparkExecutor@dhd2.quanti= find.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 re= fused: dhd2.quantifind.com/10.10.5.64:45794]



Looking in the=20 logs of the failed executor, there are some similar messages about=20 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/syste= m/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2Fspark%40ddd0.quantifind.com<= /a>%3A43068-1#772172548] was not delivered. [1] dead letters encountered. This logging can be=20 turned off or adjusted with configuration settings=20 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutd= own'.

13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transp= ort.AssociationHandle$Disassociated] from Actor[akka://sparkExecutor/deadLe= tters] 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=20 turned off or adjusted with configuration settings=20 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutd= own'.

13/10/29 11:03:53 INFO actor.LocalActorRef: Message [akka.remote.transp= ort.AssociationHandle$Disassociated] from Actor[akka://sparkExecutor/deadLe= tters] 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=20 turned off or adjusted with configuration settings=20 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutd= own'.

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

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


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

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

=A0=A0=A0 -Dspark.storage.blockManagerHeartBeatMs=3D300000
=A0=A0=A0= -Dspark.akka.frameSize=3D150
=A0=A0=A0 -Dspark.akka.timeout=3D120
= =A0=A0=A0 -Dspark.akka.askTimeout=3D30
=A0 =A0 -Dspark.akka.logLifecycle= Events=3Dtrue

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

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

thanks,
Imran



<= font color=3D"#888888">--
s



<= font color=3D"#888888">--
s




<= font color=3D"#888888">--
s





--
= s --f46d04182562b18bff04e9f6b943--