[
https://issues.apache.org/jira/browse/SPARK-10792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14980179#comment-14980179
]
Adrian Tanase commented on SPARK-10792:
---------------------------------------
Forgot to report back, seems to be fixed. However I also had to make some yarn
timeouts more aggressive, as the defaults in yarn-site.xml are too high (e.g.
takes 10 minutes for yarn to react to node being down).
The following tests passed:
* stopped nodemanager service on executor node - restarted on different machine
in ~30 secs
* stopped secondary RM - no effect
* started 6 nodes with 7 executor request - as soon as the 7th node was added
to yarn, executor was deployed
* stopped RM and node service on primary RM - secondary took over and
redeployed the app under a minute
* stopped machine with primary RM from AWS console - same
I have not corrupted the checkpoint in none of these tests - it always came
back nicely.
> Spark + YARN – executor is not re-created
> -----------------------------------------
>
> Key: SPARK-10792
> URL: https://issues.apache.org/jira/browse/SPARK-10792
> Project: Spark
> Issue Type: Bug
> Components: YARN
> Affects Versions: 1.4.0
> Environment: - centos7 deployed on AWS
> - yarn / hadoop 2.6.0-cdh5.4.2
> - spark 1.4.0 compiled with hadoop 2.6
> Reporter: Adrian Tanase
> Priority: Critical
> Attachments: Screen Shot 2015-09-21 at 1.58.28 PM.png
>
>
> We’re using spark streaming (1.4.0), deployed on AWS through yarn. It’s a
> stateful app that reads from kafka (with the new direct API) and we’re
> checkpointing to HDFS.
> During some resilience testing, we restarted one of the machines and brought
> it back online. During the offline period, the Yarn cluster would not have
> resources to re-create the missing executor.
> After starting all the services on the machine, it correctly joined the Yarn
> cluster, however the spark streaming app does not seem to notice that the
> resources are back and has not re-created the missing executor.
> The app is correctly running with 6 out of 7 executors, however it’s running
> under capacity.
> If we manually kill the driver and re-submit the app to yarn, all the sate is
> correctly recreated from checkpoint and all 7 executors are now online –
> however this seems like a brutal workaround.
> *Scenarios tested to isolate the issue:*
> The expected outcome after a machine reboot + services back is that
> processing continues on it. *FAILED* below means that processing continues in
> a reduced capacity, as the machine lost rarely re-joins as container/executor
> even if YARN sees it as healthy node.
> || No || Failure scenario || test result || data loss || Notes ||
> | 1 | Single node restart | FAILED | NO | Executor NOT redeployed when
> machine comes back and services are restarted |
> | 2 | Multi-node restart (quick succession) | FAILED | YES | If we are not
> restoring services on machines that are down, the app OR kafka OR zookeeper
> metadata gets corrupted, app crashes and can't be restarted w/o clearing
> checkpoint -> dataloss. Root cause is unhealthy cluster when too many
> machines are lost. |
> | 3 | Multi-node restart (rolling) | FAILED | NO | Same as single node
> restart, driver does not crash |
> | 4 | Graceful services restart | FAILED | NO | Behaves just like single
> node restart even if we take the time to manually stop services before
> machine reboot. |
> | 5 | Adding nodes to an incomplete cluster | SUCCESS | NO | The spark app
> will usually start even if YARN can't fullfill all the resource requests
> (e.g. 5 out of 7 nodes are up when app is started). However, when the nodes
> are added to YARN, we see that Spark deploys executors on them, as expected
> in all the scenarios. |
> | 6 | Restart executor process | PARTIAL SUCCESS | NO | 1 out of 5 attempts
> it behaves like machine restart - the rest work as expected,
> container/executor are redeployed in a matter of seconds |
> | 7 | Node restart on bigger cluster | FAILED | NO | We were trying to
> validate if the behavior is caused by maxing out the cluster and having no
> slack to redeploy a crashed node. We are still behaving like single node
> restart event with lots of extra capacity in YARN - nodes, cores and RAM. |
> *Logs for Scenario 6 – correct behavior on process restart*
> {noformat}
> 2015-09-21 11:00:11,193 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Completed container
> container_1442827158253_0004_01_000004 (state: COMPLETE, exit status: 137)
> 2015-09-21 11:00:11,193 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Container marked as failed:
> container_1442827158253_0004_01_000004. Exit status: 137. Diagnostics:
> Container killed on request. Exit code is 137
> Container exited with a non-zero exit code 137
> Killed by external signal
> ..
> (logical continuation from earlier restart attempt)
> 2015-09-21 10:33:20,658 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Will request 1 executor
> containers, each with 14 cores and 18022 MB memory including 1638 MB overhead
> 2015-09-21 10:33:20,658 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Container request (host: Any,
> capability: <memory:18022, vCores:14>)
> ..
> 2015-09-21 10:33:25,663 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Launching container
> container_1442827158253_0004_01_000012 for on host ip-10-0-1-16.ec2.internal
> 2015-09-21 10:33:25,664 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Launching ExecutorRunnable.
> driverUrl:
> akka.tcp://[email protected]:32938/user/CoarseGrainedScheduler,
> executorHostname: ip-10-0-1-16.ec2.internal
> 2015-09-21 10:33:25,664 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Received 1 containers from YARN,
> launching executors on 1 of them.
> {noformat}
> *Logs for Scenario 1 – weird resource requests / behavior on node restart*
> {noformat}
> 2015-09-21 10:36:57,352 [sparkDriver-akka.actor.default-dispatcher-31] INFO
> org.apache.spark.deploy.yarn.ApplicationMaster$AMEndpoint - Driver terminated
> or disconnected! Shutting down. ip-10-0-1-16.ec2.internal:34741
> 2015-09-21 10:36:57,352 [sparkDriver-akka.actor.default-dispatcher-24] ERROR
> org.apache.spark.scheduler.cluster.YarnClusterScheduler - Lost executor 8 on
> ip-10-0-1-16.ec2.internal: remote Rpc client disassociated
> 2015-09-21 10:36:57,352 [sparkDriver-akka.actor.default-dispatcher-24] INFO
> org.apache.spark.deploy.yarn.ApplicationMaster$AMEndpoint - Driver terminated
> or disconnected! Shutting down. ip-10-0-1-16.ec2.internal:34741
> 2015-09-21 10:36:57,352 [sparkDriver-akka.actor.default-dispatcher-31] WARN
> akka.remote.ReliableDeliverySupervisor - Association with remote system
> [akka.tcp://[email protected]:34741] has failed,
> address is now gated for [5000] ms. Reason is: [Disassociated].
> 2015-09-21 10:36:57,352 [dag-scheduler-event-loop] INFO
> org.apache.spark.scheduler.DAGScheduler - Executor lost: 8 (epoch 995)
> 2015-09-21 10:36:57,352 [sparkDriver-akka.actor.default-dispatcher-31] INFO
> org.apache.spark.storage.BlockManagerMasterEndpoint - Trying to remove
> executor 8 from BlockManagerMaster.
> 2015-09-21 10:36:57,352 [sparkDriver-akka.actor.default-dispatcher-31] INFO
> org.apache.spark.storage.BlockManagerMasterEndpoint - Removing block manager
> BlockManagerId(8, ip-10-0-1-16.ec2.internal, 35415)
> 2015-09-21 10:36:57,352 [dag-scheduler-event-loop] INFO
> org.apache.spark.storage.BlockManagerMaster - Removed 8 successfully in
> removeExecutor
> 20
> ...
> 2015-09-21 10:39:44,320 [sparkDriver-akka.actor.default-dispatcher-31] WARN
> org.apache.spark.HeartbeatReceiver - Removing executor 8 with no recent
> heartbeats: 168535 ms exceeds timeout 120000 ms
> 2015-09-21 10:39:44,320 [sparkDriver-akka.actor.default-dispatcher-31] ERROR
> org.apache.spark.scheduler.cluster.YarnClusterScheduler - Lost an executor 8
> (already removed): Executor heartbeat timed out after 168535 ms
> 2015-09-21 10:39:44,320 [kill-executor-thread] INFO
> org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend - Requesting
> to kill executor(s) 8
> 2015-09-21 10:39:44,320 [kill-executor-thread] WARN
> org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend - Executor to
> kill 8 does not exist!
> 2015-09-21 10:39:44,320 [sparkDriver-akka.actor.default-dispatcher-31] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Driver requested a total number
> of 5 executor(s).
> 2015-09-21 10:39:44,321 [sparkDriver-akka.actor.default-dispatcher-31] INFO
> org.apache.spark.deploy.yarn.ApplicationMaster$AMEndpoint - Driver requested
> to kill executor(s) .
> 2015-09-21 10:39:45,793 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Canceling requests for 0
> executor containers
> 2015-09-21 10:39:45,793 [Reporter] WARN
> org.apache.spark.deploy.yarn.YarnAllocator - Expected to find pending
> requests, but found none.
> ... every 5 seconds
> 2015-09-21 10:40:05,800 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Canceling requests for 0
> executor containers
> 2015-09-21 10:40:05,800 [Reporter] WARN
> org.apache.spark.deploy.yarn.YarnAllocator - Expected to find pending
> requests, but found none.
> ..
> 2015-09-21 10:43:55,876 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Canceling requests for 0
> executor containers
> 2015-09-21 10:43:55,876 [Reporter] WARN
> org.apache.spark.deploy.yarn.YarnAllocator - Expected to find pending
> requests, but found none.
> ...
> 2015-09-21 10:49:20,979 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Canceling requests for 0
> executor containers
> 2015-09-21 10:49:20,979 [Reporter] WARN
> org.apache.spark.deploy.yarn.YarnAllocator - Expected to find pending
> requests, but found none.
> 2015-09-21 10:49:20,980 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Completed container
> container_1442827158253_0004_01_000012 (state: COMPLETE, exit status: -100)
> 2015-09-21 10:49:20,980 [Reporter] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Container marked as failed:
> container_1442827158253_0004_01_000012. Exit status: -100. Diagnostics:
> Container released on a *lost* node
> .. done
> =======
> (ANOTHER RESTART ATTEMPT – note how it’s now requesting a total of 4 (should
> be 7!)
> 2015-09-21 10:51:28,226 [sparkDriver-akka.actor.default-dispatcher-19] INFO
> org.apache.spark.deploy.yarn.ApplicationMaster$AMEndpoint - Driver terminated
> or disconnected! Shutting down. ip-10-0-1-15.ec2.internal:34332
> 2015-09-21 10:51:28,226 [sparkDriver-akka.actor.default-dispatcher-31] ERROR
> org.apache.spark.scheduler.cluster.YarnClusterScheduler - Lost executor 1 on
> ip-10-0-1-15.ec2.internal: remote Rpc client disassociated
> 2015-09-21 10:51:28,226 [sparkDriver-akka.actor.default-dispatcher-30] WARN
> akka.remote.ReliableDeliverySupervisor - Association with remote system
> [akka.tcp://[email protected]:34332] has failed,
> address is now gated for [5000] ms. Reason is: [Disassociated].
> 2015-09-21 10:51:28,226 [sparkDriver-akka.actor.default-dispatcher-31] INFO
> org.apache.spark.deploy.yarn.ApplicationMaster$AMEndpoint - Driver terminated
> or disconnected! Shutting down. ip-10-0-1-15.ec2.internal:34332
> 2015-09-21 10:51:28,226 [dag-scheduler-event-loop] INFO
> org.apache.spark.scheduler.DAGScheduler - Executor lost: 1 (epoch 1431)
> 2015-09-21 10:51:28,226 [sparkDriver-akka.actor.default-dispatcher-32] INFO
> org.apache.spark.storage.BlockManagerMasterEndpoint - Trying to remove
> executor 1 from BlockManagerMaster.
> 2015-09-21 10:51:28,227 [sparkDriver-akka.actor.default-dispatcher-32] INFO
> org.apache.spark.storage.BlockManagerMasterEndpoint - Removing block manager
> BlockManagerId(1, ip-10-0-1-15.ec2.internal, 36311)
> 2015-09-21 10:51:28,227 [dag-scheduler-event-loop] INFO
> org.apache.spark.storage.BlockManagerMaster - Removed 1 successfully in
> removeExecutor
> ...
> 2015-09-21 10:53:44,320 [sparkDriver-akka.actor.default-dispatcher-32] WARN
> org.apache.spark.HeartbeatReceiver - Removing executor 1 with no recent
> heartbeats: 140055 ms exceeds timeout 120000 ms
> 2015-09-21 10:53:44,320 [sparkDriver-akka.actor.default-dispatcher-32] ERROR
> org.apache.spark.scheduler.cluster.YarnClusterScheduler - Lost an executor 1
> (already removed): Executor heartbeat timed out after 140055 ms
> 2015-09-21 10:53:44,320 [kill-executor-thread] INFO
> org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend - Requesting
> to kill executor(s) 1
> 2015-09-21 10:53:44,320 [kill-executor-thread] WARN
> org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend - Executor to
> kill 1 does not exist!
> 2015-09-21 10:53:44,320 [sparkDriver-akka.actor.default-dispatcher-32] INFO
> org.apache.spark.deploy.yarn.YarnAllocator - Driver requested a total number
> of 4 executor(s).
> 2015-09-21 10:53:44,321 [sparkDriver-akka.actor.default-dispatcher-32] INFO
> org.apache.spark.deploy.yarn.ApplicationMaster$AMEndpoint - Driver requested
> to kill executor(s) .
> {noformat}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]