You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Igor Berman (JIRA)" <ji...@apache.org> on 2018/01/31 17:12:00 UTC

[jira] [Created] (SPARK-23286) Partial registration to external shuffle services on Mesos

Igor Berman created SPARK-23286:
-----------------------------------

             Summary: Partial registration to external shuffle services on Mesos
                 Key: SPARK-23286
                 URL: https://issues.apache.org/jira/browse/SPARK-23286
             Project: Spark
          Issue Type: Bug
          Components: Mesos, Shuffle
    Affects Versions: 2.2.0
            Reporter: Igor Berman


As continuation of https://issues.apache.org/jira/browse/SPARK-12583

I see that registration to external shuffle services is partial only(i.e. the application is not registered to all external shuffle services where executors are running for given framework/application)

Seems like some updates either not handled or lost. Probably the first option is more relevant due to registering only when statusUpdate handles TaskStatus that equals RUNNING ([https://github.com/apache/spark/blob/cfcd746689c2b84824745fa6d327ffb584c7a17d/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala#L618)]

 

Setup:
{code:java}
spark.shuffle.service.enabled = true
spark.dynamicAllocation.enabled = true
spark.dynamicAllocation.executorIdleTimeout = 20s
spark.shuffle.service.port = 7337
spark.dynamicAllocation.maxExecutors = 50
spark.dynamicAllocation.minExecutors = 2
spark.dynamicAllocation.cachedExecutorIdleTimeout = 300s

spark.default.parallelism = 3000
spark.sql.shuffle.partitions = 3000
spark.cores.max = 400
spark.driver.maxResultSize = 8g
spark.executor.memory = 70g
spark.mesos.coarse = true

spark.worker.timeout = 150000
spark.network.timeout = 1200s
spark.executor.heartbeatInterval = 600s
spark.hadoop.cloneConf = true
spark.memory.fraction = 0.75
spark.memory.storageFraction = 0.1{code}
 

 

 

Here is trace extraction which "proves" that only 2 registration happened, however at this point several(>2) Mesos tasks(spark executors) were running
{code:java}
grep "MesosClusterScheduler\|CoarseGrainedSchedulerBackend\|MesosExternalShuffleClient\|MesosCoarseGrainedSchedulerBackend" /var/log/mycomp/myservice.log
2018-01-31 16:26:15,516 INFO [main] MesosCoarseGrainedSchedulerBackend [] - Capping the total amount of executors to 2
2018-01-31 16:26:15,520 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Received 16 resource offers.
2018-01-31 16:26:15,530 INFO [main] MesosCoarseGrainedSchedulerBackend [] - SchedulerBackend is ready for scheduling beginning after reached minRegisteredResourcesRatio: 0.0
2018-01-31 16:26:15,588 WARN [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Unable to parse into a key:value label for the task.
2018-01-31 16:26:15,667 WARN [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Unable to parse into a key:value label for the task.
2018-01-31 16:26:15,675 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Declining offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857709 with attributes: Map() mem: 178811.35 cpu: 0.0 port: List((31000,31717), (31719,32000))
2018-01-31 16:26:15,678 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Accepting offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857710 with attributes: Map() mem: 257659.7 cpu: 40.0 ports: List((31000,31565), (31567,32000)). Launching 1 Mesos tasks.
2018-01-31 16:26:15,680 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Launching Mesos task: 0 with mem: 78848.0 cpu: 40.0 ports:
2018-01-31 16:26:15,681 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Accepting offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857711 with attributes: Map() mem: 257659.35 cpu: 40.0 ports: List((31000,31484), (31486,32000)). Launching 1 Mesos tasks.
2018-01-31 16:26:15,681 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Launching Mesos task: 1 with mem: 78848.0 cpu: 40.0 ports:
2018-01-31 16:26:15,683 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Declining offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857712 with attributes: Map() mem: 178812.88 cpu: 4.0 port: List((31000,31837), (31839,32000))

2018-01-31 16:26:15,690 DEBUG [Thread-25] MesosCoarseGrainedSchedulerBackend [] - Declining offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857724 with attributes: Map() mem: 257659.35 cpu: 40.0 port: List((31000,31845), (31847,32000))
2018-01-31 16:26:18,634 INFO [Thread-27] MesosCoarseGrainedSchedulerBackend [] - Mesos task 1 is now TASK_RUNNING
2018-01-31 16:26:18,636 DEBUG [Thread-27] MesosCoarseGrainedSchedulerBackend [] - Connecting to shuffle service on slave 7b6c96d5-a734-43e6-9585-11363e63163d-S14, host sparktest007.mycomp.com, port 7337 for app 7b6c96d5-a734-43e6-9585-11363e63163d-0054
2018-01-31 16:26:18,688 INFO [Thread-28] MesosCoarseGrainedSchedulerBackend [] - Mesos task 0 is now TASK_RUNNING
2018-01-31 16:26:18,689 DEBUG [Thread-28] MesosCoarseGrainedSchedulerBackend [] - Connecting to shuffle service on slave 7b6c96d5-a734-43e6-9585-11363e63163d-S22, host sparktest011.mycomp.com, port 7337 for app 7b6c96d5-a734-43e6-9585-11363e63163d-0054
2018-01-31 16:26:18,719 INFO [shuffle-client-4-2] MesosExternalShuffleClient [] - Successfully registered app 7b6c96d5-a734-43e6-9585-11363e63163d-0054 with external shuffle service.
2018-01-31 16:26:18,719 INFO [shuffle-client-4-1] MesosExternalShuffleClient [] - Successfully registered app 7b6c96d5-a734-43e6-9585-11363e63163d-0054 with external shuffle service.
2018-01-31 16:26:21,640 DEBUG [Thread-29] MesosCoarseGrainedSchedulerBackend [] - Received 16 resource offers.
2018-01-31 16:26:21,644 DEBUG [Thread-29] MesosCoarseGrainedSchedulerBackend [] - Declining offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857725 with attributes: Map() mem: 257659.35 cpu: 40.0 port: List((31000,31377), (31379,32000))
2018-01-31 16:26:21,645 DEBUG [Thread-29] MesosCoarseGrainedSchedulerBackend [] - Declining offer: 7b6c96d5-a734-43e6-9585-11363e63163d-O857726 with attributes: Map() mem: 178811.35 cpu: 0.0 p
2018-01-31 16:26:21,744 INFO [dispatcher-event-loop-33] CoarseGrainedSchedulerBackend$DriverEndpoint [] - Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.16.123.241:33794) with ID 0
2018-01-31 16:26:21,783 INFO [dispatcher-event-loop-33] CoarseGrainedSchedulerBackend$DriverEndpoint [] - Registered executor NettyRpcEndpointRef(spark-client://Executor) (10.100.130.11:35828) with ID 1{code}
 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org