[ https://issues.apache.org/jira/browse/SPARK-30299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17076525#comment-17076525 ]
Thomas Graves commented on SPARK-30299: --------------------------------------- Note that there are other places in the code that uses executor cores which could also be wrong in standalone mode. for instance PythonRunner is using it to split memory. > Dynamic allocation with Standalone mode calculates to many executors needed > --------------------------------------------------------------------------- > > Key: SPARK-30299 > URL: https://issues.apache.org/jira/browse/SPARK-30299 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.4.4, 3.0.0 > Reporter: Thomas Graves > Priority: Major > > While I was doing some changes in the executor allocation manager, I realized > there is a bug with dynamic allocation in standalone mode. > The issue is that if you run standalone mode with the default settings where > the executor gets all the cores of the worker, spark core (allocation > manager) doesn't know the number of cores per executor to be able to > calculate how many tasks can fit on an executor. > It therefore defaults to the use the default EXECUTOR_CORES which is 1 and > thus could calculate it needs way more containers there are actually does. > For instance, I have a worker with 12 cores. That means by default when I > start an executor on it, it gets 12 cores and can fit 12 tasks. The > allocation manager would use the default of 1 core per executor and say it > needs 12 executors when it only needs 1. > The fix for this isn't trivial since it would need to know how many cores > each one has and I assume it would also need to handle heterogenous nodes. > I could start workers on nodes with different numbers of cores - one with 24 > cores and one with 16 cores. How do we estimate the number of executors in > this case. We could just choose the min of existing ones or something like > that as an estimate and it would be closer, unless of course the next > executor you got didn't actually have that. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org