Skip to content

Commit 353c30b

Browse files
jerryshaoMarcelo Vanzin
authored andcommitted
[SPARK-10790] [YARN] Fix initial executor number not set issue and consolidate the codes
This bug is introduced in [SPARK-9092](https://issues.apache.org/jira/browse/SPARK-9092), `targetExecutorNumber` should use `minExecutors` if `initialExecutors` is not set. Using 0 instead will meet the problem as mentioned in [SPARK-10790](https://issues.apache.org/jira/browse/SPARK-10790). Also consolidate and simplify some similar code snippets to keep the consistent semantics. Author: jerryshao <[email protected]> Closes #8910 from jerryshao/SPARK-10790.
1 parent d8d50ed commit 353c30b

File tree

4 files changed

+27
-40
lines changed

4 files changed

+27
-40
lines changed

yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala

Lines changed: 1 addition & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -81,25 +81,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
8181
.orNull
8282
// If dynamic allocation is enabled, start at the configured initial number of executors.
8383
// Default to minExecutors if no initialExecutors is set.
84-
if (isDynamicAllocationEnabled) {
85-
val minExecutorsConf = "spark.dynamicAllocation.minExecutors"
86-
val initialExecutorsConf = "spark.dynamicAllocation.initialExecutors"
87-
val maxExecutorsConf = "spark.dynamicAllocation.maxExecutors"
88-
val minNumExecutors = sparkConf.getInt(minExecutorsConf, 0)
89-
val initialNumExecutors = sparkConf.getInt(initialExecutorsConf, minNumExecutors)
90-
val maxNumExecutors = sparkConf.getInt(maxExecutorsConf, Integer.MAX_VALUE)
91-
92-
// If defined, initial executors must be between min and max
93-
if (initialNumExecutors < minNumExecutors || initialNumExecutors > maxNumExecutors) {
94-
throw new IllegalArgumentException(
95-
s"$initialExecutorsConf must be between $minExecutorsConf and $maxNumExecutors!")
96-
}
97-
98-
numExecutors = initialNumExecutors
99-
} else {
100-
val numExecutorsConf = "spark.executor.instances"
101-
numExecutors = sparkConf.getInt(numExecutorsConf, numExecutors)
102-
}
84+
numExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
10385
principal = Option(principal)
10486
.orElse(sparkConf.getOption("spark.yarn.principal"))
10587
.orNull

yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -89,11 +89,7 @@ private[yarn] class YarnAllocator(
8989
@volatile private var numExecutorsFailed = 0
9090

9191
@volatile private var targetNumExecutors =
92-
if (Utils.isDynamicAllocationEnabled(sparkConf)) {
93-
sparkConf.getInt("spark.dynamicAllocation.initialExecutors", 0)
94-
} else {
95-
sparkConf.getInt("spark.executor.instances", YarnSparkHadoopUtil.DEFAULT_NUMBER_EXECUTORS)
96-
}
92+
YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
9793

9894
// Executor loss reason requests that are pending - maps from executor ID for inquiry to a
9995
// list of requesters that should be responded to once we find out why the given executor

yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -314,5 +314,28 @@ object YarnSparkHadoopUtil {
314314
def getClassPathSeparator(): String = {
315315
classPathSeparatorField.get(null).asInstanceOf[String]
316316
}
317+
318+
/**
319+
* Getting the initial target number of executors depends on whether dynamic allocation is
320+
* enabled.
321+
*/
322+
def getInitialTargetExecutorNumber(conf: SparkConf): Int = {
323+
if (Utils.isDynamicAllocationEnabled(conf)) {
324+
val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0)
325+
val initialNumExecutors =
326+
conf.getInt("spark.dynamicAllocation.initialExecutors", minNumExecutors)
327+
val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", Int.MaxValue)
328+
require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors,
329+
s"initial executor number $initialNumExecutors must between min executor number" +
330+
s"$minNumExecutors and max executor number $maxNumExecutors")
331+
332+
initialNumExecutors
333+
} else {
334+
val targetNumExecutors =
335+
sys.env.get("SPARK_EXECUTOR_INSTANCES").map(_.toInt).getOrElse(DEFAULT_NUMBER_EXECUTORS)
336+
// System property can override environment variable.
337+
conf.getInt("spark.executor.instances", targetNumExecutors)
338+
}
339+
}
317340
}
318341

yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala

Lines changed: 2 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -17,21 +17,13 @@
1717

1818
package org.apache.spark.scheduler.cluster
1919

20-
import java.net.NetworkInterface
21-
2220
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
23-
24-
import scala.collection.JavaConverters._
25-
26-
import org.apache.hadoop.yarn.api.records.NodeState
27-
import org.apache.hadoop.yarn.client.api.YarnClient
2821
import org.apache.hadoop.yarn.conf.YarnConfiguration
2922

3023
import org.apache.spark.SparkContext
3124
import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
32-
import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
3325
import org.apache.spark.scheduler.TaskSchedulerImpl
34-
import org.apache.spark.util.{IntParam, Utils}
26+
import org.apache.spark.util.Utils
3527

3628
private[spark] class YarnClusterSchedulerBackend(
3729
scheduler: TaskSchedulerImpl,
@@ -40,13 +32,7 @@ private[spark] class YarnClusterSchedulerBackend(
4032

4133
override def start() {
4234
super.start()
43-
totalExpectedExecutors = DEFAULT_NUMBER_EXECUTORS
44-
if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) {
45-
totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES"))
46-
.getOrElse(totalExpectedExecutors)
47-
}
48-
// System property can override environment variable.
49-
totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors)
35+
totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sc.conf)
5036
}
5137

5238
override def applicationId(): String =

0 commit comments

Comments
 (0)