Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.internal.config

import java.util.concurrent.TimeUnit

private[spark] object Streaming {

private[spark] val STREAMING_DYN_ALLOCATION_ENABLED =
ConfigBuilder("spark.streaming.dynamicAllocation.enabled")
.booleanConf
.createWithDefault(false)

private[spark] val STREAMING_DYN_ALLOCATION_TESTING =
ConfigBuilder("spark.streaming.dynamicAllocation.testing")
.booleanConf
.createWithDefault(false)

private[spark] val STREAMING_DYN_ALLOCATION_MIN_EXECUTORS =
ConfigBuilder("spark.streaming.dynamicAllocation.minExecutors")
.intConf
.checkValue(_ > 0, "The min executor number of streaming dynamic " +
"allocation must be positive.")
.createOptional

private[spark] val STREAMING_DYN_ALLOCATION_MAX_EXECUTORS =
ConfigBuilder("spark.streaming.dynamicAllocation.maxExecutors")
.intConf
.checkValue(_ > 0, "The max executor number of streaming dynamic " +
"allocation must be positive.")
.createWithDefault(Int.MaxValue)

private[spark] val STREAMING_DYN_ALLOCATION_SCALING_INTERVAL =
ConfigBuilder("spark.streaming.dynamicAllocation.scalingInterval")
.timeConf(TimeUnit.SECONDS)
.checkValue(_ > 0, "The scaling interval of streaming dynamic " +
"allocation must be positive.")
.createWithDefault(60)

private[spark] val STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO =
ConfigBuilder("spark.streaming.dynamicAllocation.scalingUpRatio")
.doubleConf
.checkValue(_ > 0, "The scaling up ratio of streaming dynamic " +
"allocation must be positive.")
.createWithDefault(0.9)

private[spark] val STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO =
ConfigBuilder("spark.streaming.dynamicAllocation.scalingDownRatio")
.doubleConf
.checkValue(_ > 0, "The scaling down ratio of streaming dynamic " +
"allocation must be positive.")
.createWithDefault(0.3)
}
7 changes: 7 additions & 0 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config._
import org.apache.spark.internal.config.Streaming._
import org.apache.spark.internal.config.Tests.IS_TESTING
import org.apache.spark.internal.config.UI._
import org.apache.spark.internal.config.Worker._
Expand Down Expand Up @@ -2490,6 +2491,12 @@ private[spark] object Utils extends Logging {
(!isLocalMaster(conf) || conf.get(DYN_ALLOCATION_TESTING))
}

def isStreamingDynamicAllocationEnabled(conf: SparkConf): Boolean = {
val streamingDynamicAllocationEnabled = conf.get(STREAMING_DYN_ALLOCATION_ENABLED)
streamingDynamicAllocationEnabled &&
(!isLocalMaster(conf) || conf.get(STREAMING_DYN_ALLOCATION_TESTING))
}

/**
* Return the initial number of executors for dynamic allocation.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import org.apache.spark.deploy.history.HistoryServer
import org.apache.spark.deploy.yarn.config._
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.internal.config.Streaming.STREAMING_DYN_ALLOCATION_MAX_EXECUTORS
import org.apache.spark.internal.config.UI._
import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances}
import org.apache.spark.rpc._
Expand Down Expand Up @@ -100,7 +101,9 @@ private[spark] class ApplicationMaster(

private val maxNumExecutorFailures = {
val effectiveNumExecutors =
if (Utils.isDynamicAllocationEnabled(sparkConf)) {
if (Utils.isStreamingDynamicAllocationEnabled(sparkConf)) {
sparkConf.get(STREAMING_DYN_ALLOCATION_MAX_EXECUTORS)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For other reviewers -- this is the fix itself.

} else if (Utils.isDynamicAllocationEnabled(sparkConf)) {
sparkConf.get(DYN_ALLOCATION_MAX_EXECUTORS)
} else {
sparkConf.get(EXECUTOR_INSTANCES).getOrElse(0)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import scala.util.Random

import org.apache.spark.{ExecutorAllocationClient, SparkConf}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.Streaming._
import org.apache.spark.streaming.util.RecurringTimer
import org.apache.spark.util.{Clock, Utils}

Expand Down Expand Up @@ -55,17 +56,12 @@ private[streaming] class ExecutorAllocationManager(
batchDurationMs: Long,
clock: Clock) extends StreamingListener with Logging {

import ExecutorAllocationManager._

private val scalingIntervalSecs = conf.getTimeAsSeconds(
SCALING_INTERVAL_KEY,
s"${SCALING_INTERVAL_DEFAULT_SECS}s")
private val scalingUpRatio = conf.getDouble(SCALING_UP_RATIO_KEY, SCALING_UP_RATIO_DEFAULT)
private val scalingDownRatio = conf.getDouble(SCALING_DOWN_RATIO_KEY, SCALING_DOWN_RATIO_DEFAULT)
private val minNumExecutors = conf.getInt(
MIN_EXECUTORS_KEY,
math.max(1, receiverTracker.numReceivers))
private val maxNumExecutors = conf.getInt(MAX_EXECUTORS_KEY, Integer.MAX_VALUE)
private val scalingIntervalSecs = conf.get(STREAMING_DYN_ALLOCATION_SCALING_INTERVAL)
private val scalingUpRatio = conf.get(STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO)
private val scalingDownRatio = conf.get(STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO)
private val minNumExecutors = conf.get(STREAMING_DYN_ALLOCATION_MIN_EXECUTORS)
.getOrElse(math.max(1, receiverTracker.numReceivers()))
private val maxNumExecutors = conf.get(STREAMING_DYN_ALLOCATION_MAX_EXECUTORS)
private val timer = new RecurringTimer(clock, scalingIntervalSecs * 1000,
_ => manageAllocation(), "streaming-executor-allocation-manager")

Expand Down Expand Up @@ -150,34 +146,17 @@ private[streaming] class ExecutorAllocationManager(
}

private def validateSettings(): Unit = {
require(
scalingIntervalSecs > 0,
s"Config $SCALING_INTERVAL_KEY must be more than 0")

require(
scalingUpRatio > 0,
s"Config $SCALING_UP_RATIO_KEY must be more than 0")

require(
scalingDownRatio > 0,
s"Config $SCALING_DOWN_RATIO_KEY must be more than 0")

require(
minNumExecutors > 0,
s"Config $MIN_EXECUTORS_KEY must be more than 0")

require(
maxNumExecutors > 0,
s"$MAX_EXECUTORS_KEY must be more than 0")

require(
scalingUpRatio > scalingDownRatio,
s"Config $SCALING_UP_RATIO_KEY must be more than config $SCALING_DOWN_RATIO_KEY")
s"Config ${STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO.key} must be more than config " +
s"${STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO.key}")

if (conf.contains(MIN_EXECUTORS_KEY) && conf.contains(MAX_EXECUTORS_KEY)) {
if (conf.contains(STREAMING_DYN_ALLOCATION_MIN_EXECUTORS.key) &&
conf.contains(STREAMING_DYN_ALLOCATION_MAX_EXECUTORS.key)) {
require(
maxNumExecutors >= minNumExecutors,
s"Config $MAX_EXECUTORS_KEY must be more than config $MIN_EXECUTORS_KEY")
s"Config ${STREAMING_DYN_ALLOCATION_MAX_EXECUTORS.key} must be more than config " +
s"${STREAMING_DYN_ALLOCATION_MIN_EXECUTORS.key}")
}
}

Expand All @@ -190,23 +169,9 @@ private[streaming] class ExecutorAllocationManager(
}

private[streaming] object ExecutorAllocationManager extends Logging {
val ENABLED_KEY = "spark.streaming.dynamicAllocation.enabled"

val SCALING_INTERVAL_KEY = "spark.streaming.dynamicAllocation.scalingInterval"
val SCALING_INTERVAL_DEFAULT_SECS = 60

val SCALING_UP_RATIO_KEY = "spark.streaming.dynamicAllocation.scalingUpRatio"
val SCALING_UP_RATIO_DEFAULT = 0.9

val SCALING_DOWN_RATIO_KEY = "spark.streaming.dynamicAllocation.scalingDownRatio"
val SCALING_DOWN_RATIO_DEFAULT = 0.3

val MIN_EXECUTORS_KEY = "spark.streaming.dynamicAllocation.minExecutors"

val MAX_EXECUTORS_KEY = "spark.streaming.dynamicAllocation.maxExecutors"

def isDynamicAllocationEnabled(conf: SparkConf): Boolean = {
val streamingDynamicAllocationEnabled = conf.getBoolean(ENABLED_KEY, false)
val streamingDynamicAllocationEnabled = Utils.isStreamingDynamicAllocationEnabled(conf)
if (Utils.isDynamicAllocationEnabled(conf) && streamingDynamicAllocationEnabled) {
throw new IllegalArgumentException(
"""
Expand All @@ -215,8 +180,7 @@ private[streaming] object ExecutorAllocationManager extends Logging {
|false to use Dynamic Allocation in streaming.
""".stripMargin)
}
val testing = conf.getBoolean("spark.streaming.dynamicAllocation.testing", false)
streamingDynamicAllocationEnabled && (!Utils.isLocalMaster(conf) || testing)
streamingDynamicAllocationEnabled
}

def createIfEnabled(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,15 +26,14 @@ import org.scalatest.time.SpanSugar._

import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkFunSuite}
import org.apache.spark.internal.config.{DYN_ALLOCATION_ENABLED, DYN_ALLOCATION_TESTING}
import org.apache.spark.internal.config.Streaming._
import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext}
import org.apache.spark.util.{ManualClock, Utils}


class ExecutorAllocationManagerSuite extends SparkFunSuite
with BeforeAndAfter with BeforeAndAfterAll with MockitoSugar with PrivateMethodTester {

import ExecutorAllocationManager._

private val batchDurationMillis = 1000L
private var allocationClient: ExecutorAllocationClient = null
private var clock: StreamManualClock = null
Expand All @@ -58,7 +57,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite
reset(allocationClient)
when(allocationClient.getExecutorIds()).thenReturn(Seq("1", "2"))
addBatchProcTime(allocationManager, batchProcTimeMs.toLong)
val advancedTime = SCALING_INTERVAL_DEFAULT_SECS * 1000 + 1
val advancedTime = STREAMING_DYN_ALLOCATION_SCALING_INTERVAL.defaultValue.get * 1000 + 1
val expectedWaitTime = clock.getTimeMillis() + advancedTime
clock.advance(advancedTime)
// Make sure ExecutorAllocationManager.manageAllocation is called
Expand Down Expand Up @@ -101,25 +100,29 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite
}

// Batch proc time slightly more than the scale up ratio, should increase allocation by 1
addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_UP_RATIO_DEFAULT + 1) {
addBatchProcTimeAndVerifyAllocation(
batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO.defaultValue.get + 1) {
verifyTotalRequestedExecs(Some(3))
verifyKilledExec(None)
}

// Batch proc time slightly less than the scale up ratio, should not change allocation
addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_UP_RATIO_DEFAULT - 1) {
addBatchProcTimeAndVerifyAllocation(
batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO.defaultValue.get - 1) {
verifyTotalRequestedExecs(None)
verifyKilledExec(None)
}

// Batch proc time slightly more than the scale down ratio, should not change allocation
addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_DOWN_RATIO_DEFAULT + 1) {
addBatchProcTimeAndVerifyAllocation(
batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO.defaultValue.get + 1) {
verifyTotalRequestedExecs(None)
verifyKilledExec(None)
}

// Batch proc time slightly more than the scale down ratio, should not change allocation
addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_DOWN_RATIO_DEFAULT - 1) {
addBatchProcTimeAndVerifyAllocation(
batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO.defaultValue.get - 1) {
verifyTotalRequestedExecs(None)
verifyKilledExec(Some("2"))
}
Expand Down