diff --git a/pom.xml b/pom.xml
index 3be39553b0b73..4acd47638c02b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1983,13 +1983,12 @@
maven-source-plugin
2.4
- false
+ true
attach-sources
- jar-no-fork
test-jar-no-fork
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 54f62e6b723ac..1165061db21e3 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -81,25 +81,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
.orNull
// If dynamic allocation is enabled, start at the configured initial number of executors.
// Default to minExecutors if no initialExecutors is set.
- if (isDynamicAllocationEnabled) {
- val minExecutorsConf = "spark.dynamicAllocation.minExecutors"
- val initialExecutorsConf = "spark.dynamicAllocation.initialExecutors"
- val maxExecutorsConf = "spark.dynamicAllocation.maxExecutors"
- val minNumExecutors = sparkConf.getInt(minExecutorsConf, 0)
- val initialNumExecutors = sparkConf.getInt(initialExecutorsConf, minNumExecutors)
- val maxNumExecutors = sparkConf.getInt(maxExecutorsConf, Integer.MAX_VALUE)
-
- // If defined, initial executors must be between min and max
- if (initialNumExecutors < minNumExecutors || initialNumExecutors > maxNumExecutors) {
- throw new IllegalArgumentException(
- s"$initialExecutorsConf must be between $minExecutorsConf and $maxNumExecutors!")
- }
-
- numExecutors = initialNumExecutors
- } else {
- val numExecutorsConf = "spark.executor.instances"
- numExecutors = sparkConf.getInt(numExecutorsConf, numExecutors)
- }
+ numExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
principal = Option(principal)
.orElse(sparkConf.getOption("spark.yarn.principal"))
.orNull
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index ccf753e69f4b6..6a02848498a04 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -89,11 +89,7 @@ private[yarn] class YarnAllocator(
@volatile private var numExecutorsFailed = 0
@volatile private var targetNumExecutors =
- if (Utils.isDynamicAllocationEnabled(sparkConf)) {
- sparkConf.getInt("spark.dynamicAllocation.initialExecutors", 0)
- } else {
- sparkConf.getInt("spark.executor.instances", YarnSparkHadoopUtil.DEFAULT_NUMBER_EXECUTORS)
- }
+ YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
// Keep track of which container is running which executor to remove the executors later
// Visible for testing.
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index 68d01c17ef720..d7e7febbb2724 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -278,5 +278,28 @@ object YarnSparkHadoopUtil {
def getClassPathSeparator(): String = {
classPathSeparatorField.get(null).asInstanceOf[String]
}
+
+ /**
+ * Getting the initial target number of executors depends on whether dynamic allocation is
+ * enabled.
+ */
+ def getInitialTargetExecutorNumber(conf: SparkConf): Int = {
+ if (Utils.isDynamicAllocationEnabled(conf)) {
+ val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0)
+ val initialNumExecutors =
+ conf.getInt("spark.dynamicAllocation.initialExecutors", minNumExecutors)
+ val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", Int.MaxValue)
+ require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors,
+ s"initial executor number $initialNumExecutors must between min executor number" +
+ s"$minNumExecutors and max executor number $maxNumExecutors")
+
+ initialNumExecutors
+ } else {
+ val targetNumExecutors =
+ sys.env.get("SPARK_EXECUTOR_INSTANCES").map(_.toInt).getOrElse(DEFAULT_NUMBER_EXECUTORS)
+ // System property can override environment variable.
+ conf.getInt("spark.executor.instances", targetNumExecutors)
+ }
+ }
}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
index 1aed5a1675075..50b699f11b21c 100644
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
+++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
@@ -17,21 +17,13 @@
package org.apache.spark.scheduler.cluster
-import java.net.NetworkInterface
-
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.yarn.api.records.NodeState
-import org.apache.hadoop.yarn.client.api.YarnClient
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.spark.SparkContext
import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
-import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
import org.apache.spark.scheduler.TaskSchedulerImpl
-import org.apache.spark.util.{IntParam, Utils}
+import org.apache.spark.util.Utils
private[spark] class YarnClusterSchedulerBackend(
scheduler: TaskSchedulerImpl,
@@ -40,13 +32,7 @@ private[spark] class YarnClusterSchedulerBackend(
override def start() {
super.start()
- totalExpectedExecutors = DEFAULT_NUMBER_EXECUTORS
- if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) {
- totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES"))
- .getOrElse(totalExpectedExecutors)
- }
- // System property can override environment variable.
- totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors)
+ totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sc.conf)
}
override def applicationId(): String =