Spark 推测执行

Spark 推测执行是一种优化技术。

在Spark中,可以通过推测执行,即Speculative Execution,来识别并在其他节点的Executor上重启某些运行缓慢的Task,并行处理同样的数据,谁先完成就用谁的结果,并将另一个未完成的Task Kill掉,从而加快Task处理速度。适用于某些Spark任务中部分Task被hang住或运行缓慢,从而拖慢了整个任务运行速度的场景。

注意:

    1. 不是所有运行缓慢的Spark任务,都可以用推测执行来解决。

    2. 使用推测执行时应谨慎。需要合适的场景、合适的参数,参数不合理可能会导致大量推测执行Task占用资源。

    3. 如Spark Streaming写Kafka缓慢,若启用推测执行,可能会导致数据重复。

     4. 被推测的Task不会再次被推测。

Spark推测执行参数

spark.speculation :默认false。是否开启推测执行。

spark.speculation.interval :默认100ms。多久检查一次要推测执行的Task。

spark.speculation.multiplier :默认1.5。一个Stage中,运行时间比成功完成的Task的运行时间的中位数还慢1.5倍的Task才可能会被推测执行。

spark.speculation.quantile: 默认0.75。推测的分位数。即一个Stage中,至少要完成75%的Task才开始推测。

Spark推测执行源码解析

源码解析

  /**
    * TaskScheduleImpl在启动时,会判断是否启动Task的推测执行。
    */
  override def start() {
    backend.start()
    if (!isLocal && conf.getBoolean("spark.speculation", false)) {
      logInfo("Starting speculative execution thread")
      // scheduleWithFixedDelay 位于`java.util.concurrent.ScheduledExecutorService#scheduleWithFixedDelay`
      // scheduleWithFixedDelay 指的是系统启动等待`第一个SPECULATION_INTERVAL_MS 时间后`,开始执行定时任务,每隔`第二个SPECULATION_INTERVAL_MS 时间`执行一次。
      // SPECULATION_INTERVAL_MS 可通过`spark.speculation.interval`参数设置
      speculationScheduler.scheduleWithFixedDelay(new Runnable {
        override def run(): Unit = Utils.tryOrStopSparkContext(sc) {
          // 检查需要推测执行的Task
          checkSpeculatableTasks()
        }
      }, SPECULATION_INTERVAL_MS, SPECULATION_INTERVAL_MS, TimeUnit.MILLISECONDS)
    }
  }

如果开启Spark推测执行(即设置参数spark.speculation=true),且不是Local模式运行,则TaskScheduleImpl在启动spark.speculation.interval(即上述第一个SPECULATION_INTERVAL_MS)时间后,会每隔spark.speculation.interval(即上述第二个SPECULATION_INTERVAL_MS)时间启动一个线程去检查需要推测执行的Task。

点击checkSpeculatableTasks()方法,跳转到org.apache.spark.scheduler.checkSpeculatableTasks,如下代码:
 

 def checkSpeculatableTasks() {
    var shouldRevive = false
    synchronized {
      // MIN_TIME_TO_SPECULATION 在原始副本运行至少这段时间后,才会启动任务的重复副本。 
      shouldRevive = rootPool.checkSpeculatableTasks(MIN_TIME_TO_SPECULATION)
    }
    if (shouldRevive) {
      // 如果有需要推测执行的Task,则SchedulerBackend向ApplicationMaster发送reviveOffers消息,获取集群中可用的executor列表,发起task
      backend.reviveOffers()
    }
  }

可以看到,该方法内部调用的是rootPool.checkSpeculatableTasks(MIN_TIME_TO_SPECULATION),如下代码:

  override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = {
    var shouldRevive = false
    //schedulableQueue是ConcurrentLinkedQueue[Schedulable]类型,而Schedulable Trait有两种类型的调度实体:Pool、TaskSetManager
    for (schedulable <- schedulableQueue.asScala) {
      shouldRevive |= schedulable.checkSpeculatableTasks(minTimeToSpeculation)
    }
    shouldRevive
  }

可以看到,最终调用的是schedulable.checkSpeculatableTasks(minTimeToSpeculation)方法。

schedulable是schedulableQueue中的对象,schedulableQueue是ConcurrentLinkedQueue[Schedulable]类型,而Schedulable Trait有两种类型的调度实体:Pool、TaskSetManager。

通过查看org.apache.spark.scheduler.TaskSetManager#checkSpeculatableTasks方法可看到真正检测推测Task的逻辑。如下:
 

//真正检测推测执行Task的逻辑
  override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = {
    // Can't speculate if we only have one task, and no need to speculate if the task set is a
    // zombie or is from a barrier stage.
    if (isZombie || isBarrier || numTasks == 1) {
      return false
    }
    var foundTasks = false
    // minFinishedForSpeculation=SPECULATION_QUANTILE * numTasks
    // SPECULATION_QUANTILE即spark.speculation.quantile
    // numTasks即某个Stage中Taskset的任务总数。
    val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt
    logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation)

    // 1)已经成功的Task数必须要大于等于`spark.speculation.quantile * numTasks`,才开始处理这个TaskSet
    if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) {
      val time = clock.getTimeMillis()
      // medianDuration: 已经成功的Task的运行时间的中位数
      // threshold=max(SPECULATION_MULTIPLIER * medianDuration, minTimeToSpeculation)
      // SPECULATION_MULTIPLIER:即spark.speculation.multiplier
      val medianDuration = successfulTaskDurations.median
      val threshold = max(SPECULATION_MULTIPLIER * medianDuration, minTimeToSpeculation)
      // TODO: Threshold should also look at standard deviation of task durations and have a lower
      // bound based on that.
      logDebug("Task length threshold for speculation: " + threshold)
      // 2)遍历TaskSet中的每一个Task
      for (tid <- runningTasksSet) {
        val info = taskInfos(tid)
        val index = info.index
        // 3)如果还未运行成功 且 正在执行 且 运行时间已经超过threshold 且 当前不是推测运行的Task
        // 就将该Task取出放到需要推测执行的列表中
        if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold &&
          !speculatableTasks.contains(index)) {
          logInfo(
            "Marking task %d in stage %s (on %s) as speculatable because it ran more than %.0f ms"
              .format(index, taskSet.id, info.host, threshold))
          speculatableTasks += index
          // 4)最终由DAGScheduler将Task提交到待执行的队列中,后台线程将对提交的任务进行处理
          sched.dagScheduler.speculativeTaskSubmitted(tasks(index))
          foundTasks = true
        }
      }
    }
    foundTasks
  }

检测推测Task的大致流程

Spark推测执行示例

代码示例

package com.bigdata.spark

import org.apache.spark.TaskContext
import org.apache.spark.sql.SparkSession
import org.slf4j.LoggerFactory

/**
  * Author: Wang Pei
  * License: Copyright(c) Pei.Wang
  * Summary: 
  *   Spark推测执行
  */
object SparkSpeculative {
  def main(args: Array[String]): Unit = {

    @transient lazy val logger = LoggerFactory.getLogger(this.getClass)

    val spark=SparkSession.builder()
      //启用Spark推测执行
      .config("spark.speculation",true)
      .config("spark.speculation.interval",1000)
      .config("spark.speculation.multiplier",1.5)
      .config("spark.speculation.quantile",0.10)
      .getOrCreate()


    logger.info("开始处理.........................................")

     //设置5个并行度,一个Stage中,5个Task同时运行
     //为保证5个Task同时运行,Spark Submit提交任务时给5个核
     //这样,方便观察第4个Task被推测执行
    spark.sparkContext.parallelize(0 to 50,5)
      .foreach(item=>{
        if(item ==38){Thread.sleep(200000)}
        val taskContext = TaskContext.get()
        val stageId = taskContext.stageId()
        val taskAttemptId = taskContext.taskAttemptId()
        logger.info(s"当前Stage:${stageId},Task:${taskAttemptId},打印的数字..............${item}..................")
      })

    logger.info("处理完成.........................................")
  }
}

任务提交

/data/apps/spark-2.4.0-bin-2.7.3.2.6.5.3-10/bin/spark-submit \
    --master yarn \
    --deploy-mode cluster \
    --driver-memory 1g \
    --executor-memory 1g \
    --executor-cores  1 \
    --num-executors  5 \
    --queue offline \
    --name SparkSpeculative \
    --class com.bigdata.spark.SparkSpeculative \
    bigdata_spark.jar

Yarn日志查看

在Yarn上可以看到如下日志:

19/03/31 04:21:39 INFO scheduler.TaskSetManager: Finished task 2.0 in stage 0.0 (TID 2) in 318 ms on x.x.x.x (executor 4) (1/5)
19/03/31 04:21:39 INFO scheduler.TaskSetManager: Finished task 4.0 in stage 0.0 (TID 4) in 321 ms on x.x.x.x (executor 2) (2/5)
19/03/31 04:21:39 INFO scheduler.TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 338 ms on x.x.x.x (executor 1) (3/5)
19/03/31 04:21:39 INFO scheduler.TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 327 ms on x.x.x.x (executor 3) (4/5)
#task 3被标记为推测执行
19/03/31 04:21:40 INFO scheduler.TaskSetManager: Marking task 3 in stage 0.0 (on x.x.x.x) as speculatable because it ran more than 486 ms
#启动task 3的推测执行task(taskID=5)
19/03/31 04:21:40 INFO scheduler.TaskSetManager: Starting task 3.1 in stage 0.0 (TID 5, x.x.x.x, executor 3, partition 3, PROCESS_LOCAL, 7855 bytes)
#kill掉task 3的推测执行task(taskID=5),由于原来的task已经成功
19/03/31 04:24:59 INFO scheduler.TaskSetManager: Killing attempt 1 for task 3.1 in stage 0.0 (TID 5) on x.x.x.x as the attempt 0 succeeded on x.x.x.x
19/03/31 04:24:59 INFO scheduler.TaskSetManager: Finished task 3.0 in stage 0.0 (TID 3) in 200311 ms on x.x.x.x (executor 5) (5/5)

Spark WebUI查看

在Spark WebUI上可以看到如下结果:

  • 0
    点赞
  • 3
    收藏
    觉得还不错? 一键收藏
  • 0
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值