@@ -25,7 +25,7 @@ import scala.util.control.{ControlThrowable, NonFatal}
25
25
26
26
import com .codahale .metrics .{Gauge , MetricRegistry }
27
27
28
- import org .apache .spark .internal .Logging
28
+ import org .apache .spark .internal .{ config , Logging }
29
29
import org .apache .spark .internal .config ._
30
30
import org .apache .spark .metrics .source .Source
31
31
import org .apache .spark .scheduler ._
@@ -118,6 +118,8 @@ private[spark] class ExecutorAllocationManager(
118
118
private val inactiveShuffleExecutorIdleTimeoutS = conf.getTimeAsSeconds(
119
119
" spark.dynamicAllocation.inactiveShuffleExecutorIdleTimeout" , s " ${Integer .MAX_VALUE }s " )
120
120
121
+ private val externalShuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED )
122
+
121
123
// During testing, the methods to actually kill and add executors are mocked out
122
124
private val testing = conf.getBoolean(" spark.dynamicAllocation.testing" , false )
123
125
@@ -599,7 +601,7 @@ private[spark] class ExecutorAllocationManager(
599
601
*/
600
602
private def onExecutorIdle (executorId : String ): Unit = synchronized {
601
603
if (executorIds.contains(executorId)) {
602
- val hasActiveShuffleBlocks =
604
+ val hasActiveShuffleBlocks = ! externalShuffleServiceEnabled &&
603
605
mapOutputTracker.hasOutputsOnExecutor(executorId, activeOnly = true )
604
606
if (! removeTimes.contains(executorId)
605
607
&& ! executorsPendingToRemove.contains(executorId)
@@ -608,7 +610,8 @@ private[spark] class ExecutorAllocationManager(
608
610
// blocks we are concerned with are reported to the driver. Note that this
609
611
// does not include broadcast blocks.
610
612
val hasCachedBlocks = blockManagerMaster.hasCachedBlocks(executorId)
611
- val hasAnyShuffleBlocks = mapOutputTracker.hasOutputsOnExecutor(executorId)
613
+ val hasAnyShuffleBlocks =
614
+ ! externalShuffleServiceEnabled && mapOutputTracker.hasOutputsOnExecutor(executorId)
612
615
val now = clock.getTimeMillis()
613
616
614
617
// Use the maximum of all the timeouts that apply.
0 commit comments