[ https://issues.apache.org/jira/browse/SPARK-36964?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
ASF GitHub Bot updated SPARK-36964: ----------------------------------- Labels: pull-request-available (was: ) > Reuse CachedDNSToSwitchMapping for yarn container requests > ----------------------------------------------------------- > > Key: SPARK-36964 > URL: https://issues.apache.org/jira/browse/SPARK-36964 > Project: Spark > Issue Type: Improvement > Components: Spark Core, YARN > Affects Versions: 3.0.3, 3.1.2, 3.2.0, 3.3.0 > Reporter: gaoyajun02 > Priority: Major > Labels: pull-request-available > > Similar to SPARK-13704, In some cases, YarnAllocator add container requests > with locality preference can be expensive, it may call the topology script > for rack awareness. > When submit a very large job in a very large Yarn cluster, the topology > script may take signifiant time to run. And this blocks receiving > YarnSchedulerBackend's RequestExecutors rpc calls, This request comes from > spark dynamic executor allocation thread, which may blocks the > ExecutorAllocationListener, and then result in executorManagement queue > backlog. > > Some logs: > {code:java} > 21/09/29 12:04:35 INFO spark-dynamic-executor-allocation > ExecutorAllocationManager: Error reaching cluster manager.21/09/29 12:04:35 > INFO spark-dynamic-executor-allocation ExecutorAllocationManager: Error > reaching cluster manager.org.apache.spark.rpc.RpcTimeoutException: Futures > timed out after [120 seconds]. This timeout is controlled by > spark.rpc.askTimeout at > org.apache.spark.rpc.RpcTimeout.org$apache$spark$rpc$RpcTimeout$$createRpcTimeoutException(RpcTimeout.scala:47) > at > org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:62) > at > org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:58) > at > scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38) > at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:76) at > org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.requestTotalExecutors(CoarseGrainedSchedulerBackend.scala:839) > at > org.apache.spark.ExecutorAllocationManager.addExecutors(ExecutorAllocationManager.scala:411) > at > org.apache.spark.ExecutorAllocationManager.updateAndSyncNumExecutorsTarget(ExecutorAllocationManager.scala:361) > at > org.apache.spark.ExecutorAllocationManager.org$apache$spark$ExecutorAllocationManager$$schedule(ExecutorAllocationManager.scala:316) > at > org.apache.spark.ExecutorAllocationManager$$anon$1.run(ExecutorAllocationManager.scala:227) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745)Caused by: > java.util.concurrent.TimeoutException: Futures timed out after [120 seconds] > at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) at > scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:263) at > org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:294) at > org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75) ... 12 > more21/09/29 12:04:35 WARN spark-dynamic-executor-allocation > ExecutorAllocationManager: Unable to reach the cluster manager to request > 1922 total executors! > 21/09/29 12:04:35 INFO spark-dynamic-executor-allocation > ExecutorAllocationManager: Error reaching cluster manager.21/09/29 12:04:35 > INFO spark-dynamic-executor-allocation ExecutorAllocationManager: Error > reaching cluster manager.org.apache.spark.rpc.RpcTimeoutException: Futures > timed out after [120 seconds]. This timeout is controlled by > spark.rpc.askTimeout at > org.apache.spark.rpc.RpcTimeout.org$apache$spark$rpc$RpcTimeout$$createRpcTimeoutException(RpcTimeout.scala:47) > at > org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:62) > at > org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:58) > at > scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38) > at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:76) at > org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.requestTotalExecutors(CoarseGrainedSchedulerBackend.scala:839) > at > org.apache.spark.ExecutorAllocationManager.addExecutors(ExecutorAllocationManager.scala:411) > at > org.apache.spark.ExecutorAllocationManager.updateAndSyncNumExecutorsTarget(ExecutorAllocationManager.scala:361) > at > org.apache.spark.ExecutorAllocationManager.org$apache$spark$ExecutorAllocationManager$$schedule(ExecutorAllocationManager.scala:316) > at > org.apache.spark.ExecutorAllocationManager$$anon$1.run(ExecutorAllocationManager.scala:227) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745)Caused by: > java.util.concurrent.TimeoutException: Futures timed out after [120 seconds] > at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) at > scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:263) at > org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:294) at > org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75) ... 12 > more21/09/29 12:04:35 WARN spark-dynamic-executor-allocation > ExecutorAllocationManager: Unable to reach the cluster manager to request > 1922 total executors! > 21/09/29 12:02:49 ERROR dag-scheduler-event-loop AsyncEventQueue: Dropping > event from queue executorManagement. This likely means one of the listeners > is too slow and cannot keep up with the rate at which tasks are being started > by the scheduler. > 21/09/29 12:02:49 WARN dag-scheduler-event-loop AsyncEventQueue: Dropped 1 > events from executorManagement since the application started. > 21/09/29 12:02:55 INFO spark-listener-group-eventLog AsyncEventQueue: Process > of event > SparkListenerExecutorAdded(1632888172920,543,org.apache.spark.scheduler.cluster.ExecutorData@8cfab8f5,None) > by listener EventLoggingListener took 3.037686034s. > 21/09/29 12:03:03 INFO spark-listener-group-eventLog AsyncEventQueue: Process > of event SparkListenerBlockManagerAdded(1632888181779,BlockManagerId(1359, > --, 57233, None),2704696934,Some(2704696934),Some(0)) by listener > EventLoggingListener took 1.462598355s. > 21/09/29 12:03:49 WARN dispatcher-BlockManagerMaster AsyncEventQueue: Dropped > 74388 events from executorManagement since Wed Sep 29 12:02:49 CST 2021. > 21/09/29 12:04:35 INFO spark-listener-group-executorManagement > AsyncEventQueue: Process of event > SparkListenerStageSubmitted(org.apache.spark.scheduler.StageInfo@52f810ad,{...}) > by listener ExecutorAllocationListener took 116.526408932s. > 21/09/29 12:04:49 WARN heartbeat-receiver-event-loop-thread AsyncEventQueue: > Dropped 18892 events from executorManagement since Wed Sep 29 12:03:49 CST > 2021. > 21/09/29 12:05:49 WARN dispatcher-BlockManagerMaster AsyncEventQueue: Dropped > 19397 events from executorManagement since Wed Sep 29 12:04:49 CST 2021. > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org