Spark job提交过程

本文基于spark2.11

1. 前言

1.1 基本概念

  1. RDD
    关于RDD已经有很多文章了,可以参考一下理解Spark的核心RDD

  2. 依赖
    依赖分为窄依赖和宽依赖,下图描述了两种依赖(图片出自spark窄依赖和宽依赖)

    20160913233559680.jpeg

    从途中可以看出,窄依赖中,每一个上游RDD中的分区只会被一个下游分区依赖。而宽依赖上游RDD中的分区则可能被多个下游分区依赖。宽依赖往往意味者shuffle操作

  3. shuffle
    窄依赖中,由于只存在n - 1(n>=1)的依赖关系,分区上的数据可以像流水线一样一道道应用计算,大多时候不需要移动数据。宽依赖,又叫shuffle dependency,由于分区包含下游多个分区的数据,需要将数据移动到对应分区,这个过程称为shuffle

  4. Stage
    RDD DAG静态的描述了数据转换与依赖关系,action触发job提交时RDD DAG会被首先被划分以stage,stage划分的边界是宽依赖,也就是被划分到一个stage之内的rdd只存在窄依赖,stage之间是宽依赖。
    有两种stage:ShuffleMapStage,ResultStage。一个job中只有一个ResultStage,是job运行的最后阶段,收集结果。

  5. Task
    task定义了计算任务,一个分区一个task,task根据划分好的stage生成。
    有两种类型的task:ShuffleMapTask和ResultTask,和stage对应。

spark基于RDD上有两种操作transformation和action(见spark programming-guide),transformation(map、reduceByKey这种)使得RDD转换成新的RDD,action(foreach,top这种)则产生会触发一个新的job并提交,并产生以及收集job运行的结果。

下面的代码:

def main(args:Array[String]){
    val sparkConf = new SparkConf().setAppName("Log Query")
    val sc = new SparkContext(sparkConf)
    val lines = sc.textFile("README.md",3)
    val words = lines.flatMap(line => line.split(" "))
    val wordOne = words.map(word => (word,1))
    val wordCount = wordOne.reduceByKey(_ + _,3)
    wordCount.foreach(println)
    val resultAsArry = wordCount.collect()
}

有两个action:foreach和collect,因此会提交两个job,但是这两个job有共享了几个RDD。一个job提交会做一下几件事:

  1. DAGScheduler划分Stage
    有两种Stage:ShuffleMapStage和ResultStage,前者划分以ShuffleDependency为边界,创建时会根据RDD往前回溯到源头,然后从源头往下创建stage。后者job最后一阶段,在所有上游ShuffleMapStage包含的任务(ShuffleMapTask)完成后收集结果
  2. DAGScheduler提交Stage
    虽说最先提交ResultStage,但是提交时会追溯上游是否有未完成的Stage,直到找到所有不依赖任何Stage或者其依赖的Stage全部完成的Stage,然后提交。意味着对各stage可能并行提交。
  3. DAGScheduler根据Stage创建任务
    这一步其实是包含在提交Stage当中的。Stage是一种静态的概念,最终运行在集群中的是task,对应ShuffleMapStage和ResultStage存在两种task:ShuffleMapTask和ResultTask。一个ShuffleMapStage包含若干窄依赖RDD组成,一个RDD又由若干partition组成,task运行在每一个partition之上,也就是说会根据ShuffleMapStage创建出多个task。
  4. 提交任务
    根据ShuffleMapStage创建多个ShuffleMapTask之后,调用TaskScheduler开始调度任务。
  5. TaskScheduler调度任务
    TaskScheduler根据任务的preferedLocation寻找合适的executor,然后将任务信息包装好发送LaunchTask到executor,让executor执行任务。
  6. Executor执行任务
    executor运行在worker之上,接受到taskScheduler的LaunchTask消息后,启动任务的执行。
  7. 任务状态汇报
    任务运行信息会汇报到TaskScheduler,TaskScheduler则会汇报给DAGScheduler,DAGScheduler根据任务状态作出处理(stage中所有任务完成如提交子stage,读取上游数据失败重新提交stage等)

2. 划分Stage

RDD上的action操作触发job的提交,提交之前会完成stage的划分,一个stage可能包含一连串的RDD之间的转换,stage的边界就是两个RDD之间的shuffle依赖,以上面代码为例,wordOne.reduceByKey使得wordCount和wordOne之间产生shuffle依赖,下图便是上述代码产生的RDD DAG stage的划分之后的样子

wordCount.foreach(println)这类action操作触发job的提交,经过一系列调用进入到DAGScheduler的如下方法中:

def submitJob[T, U](
      rdd: RDD[T],
      func: (TaskContext, Iterator[T]) => U,
      partitions: Seq[Int],
      callSite: CallSite,
      resultHandler: (Int, U) => Unit,
      properties: Properties): JobWaiter[U] = {
    // Check to make sure we are not launching a task on a partition that does not exist.
    val maxPartitions = rdd.partitions.length
    partitions.find(p => p >= maxPartitions || p < 0).foreach { p =>
      throw new IllegalArgumentException(
        "Attempting to access a non-existent partition: " + p + ". " +
          "Total number of partitions: " + maxPartitions)
    }

    // 每个action触发一个job,每个job一个唯一的id
    val jobId = nextJobId.getAndIncrement()
    if (partitions.size == 0) {
      // Return immediately if the job is running 0 tasks
      return new JobWaiter[U](this, jobId, 0, resultHandler)
    }

    assert(partitions.size > 0)
    val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
    val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
    // 向事件循环中发送一个JobSubmitted的消息
    // 消息包含了rdd,jobid,partitions等信息
    eventProcessLoop.post(JobSubmitted(
      jobId, rdd, func2, partitions.toArray, callSite, waiter,
      SerializationUtils.clone(properties)))
    waiter
  }

参数解释

  1. 上面方法中参数rdd即代码中val wordCount对应的rdd
  2. 参数func,由println包装成的,下文会提到,这个func只会作用于ResultStage。
  3. partiitions,rdd包含的分区
  4. resulthandler,返回结果时回调

上面代码中并未提交job,而是发送JobSubmited消息给eventProcessLoop,由其异步的提交job。下面是eventProcessLoop的类DAGSchedulerEventProcessLoop处理接收到的消息的代码逻辑:

private def doOnReceive(event: DAGSchedulerEvent): Unit = event match {
    case JobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) =>
      dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties)
   // 此处省略了其它情况
    case ...
    
  }

接受到JobShumitted消息后,调用dagScheduler.handlerJobSummited提交job。下面是handleJobSubmitted的核心代码:

private[scheduler] def handleJobSubmitted(jobId: Int,
      finalRDD: RDD[_],
      func: (TaskContext, Iterator[_]) => _,
      partitions: Array[Int],
      callSite: CallSite,
      listener: JobListener,
      properties: Properties) {
    var finalStage: ResultStage = null
    try {
      // New stage creation may throw an exception if, for example, jobs are run on a
      // HadoopRDD whose underlying HDFS files have been deleted.
      finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite)
    } catch {
        ...
        return
    }

    val job = new ActiveJob(jobId, finalStage, callSite, listener, properties)
   ...
    logInfo("Parents of final stage: " + finalStage.parents)
    logInfo("Missing parents: " + getMissingParentStages(finalStage))

    val jobSubmissionTime = clock.getTimeMillis()
    jobIdToActiveJob(jobId) = job
    activeJobs += job
    finalStage.setActiveJob(job)
    val stageIds = jobIdToStageIds(jobId).toArray
    val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo))
    listenerBus.post(
      SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties))
    submitStage(finalStage)
  }

上述代码最后调用submitStage提交stage。在此之前需要创建stage,上面方法中有两个地方可能会创建stage。

  1. createResultStage
  2. getMissingParentStages

前面提到Stage分为两种:ResultStage和ShuffleMapStage,对于一个job而言,ResultStage是其最后阶段,收集job运行的结果,一个job对应的RDD DAG划分中,只存在一个ResultStage和多个ShuffleMapStage。

上面1中createResultStage会创建ResultStage,但是ResultStage,创建过程中会判断当前stage是否存在依赖上游stage,如果存在就会一直往上游追溯,从上至下创建。每一个stage有一个id,创建出来的stage根据id缓存,避免重复创建。

上面2中getMissingParentStages,则会在当前stage有上游依赖时递归的创建所有的上游依赖。

下图描述了1,2两个方法的调用图,

creatResultStage               getMissingParentStages
           |                          |
           |__________________________|
           v
getOrCreateParentStages <---------------------|
          |                                   |
          v                                   |
getOrCreateShuffleMapStage                    |
         |  当前以及所有存在的上游stage都要创建     |                                    |
         v                                    |
createShuffleMapStage-------------------------| 尝试创建上游stage,然后创建自己
  1. createResultStage
 private def createResultStage(
      rdd: RDD[_],
      func: (TaskContext, Iterator[_]) => _,
      partitions: Array[Int],
      jobId: Int,
      callSite: CallSite): ResultStage = {
    val parents = getOrCreateParentStages(rdd, jobId)
    val id = nextStageId.getAndIncrement()
    val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite)
    stageIdToStage(id) = stage
    updateJobIdStageIdMaps(jobId, stage)
    stage
  }
  • 上述代码中,先是调用getOrCreateParentStages,递归的创建所有上游的stage
  • stageId是ResultStage的id,这里是递增的,但是下面说道ShuffleMapStage的id则是shuffleid
  1. getOrCreateParentStages
    方法代码如下:
private def getOrCreateParentStages(rdd: RDD[_], firstJobId: Int): List[Stage] = {
    getShuffleDependencies(rdd).map { shuffleDep =>
      getOrCreateShuffleMapStage(shuffleDep, firstJobId)
    }.toList
  }

getShuffleDependencies(rdd),一直追溯rdd的依赖直到依赖类型为ShuffleDenpendency,这个方法实现了广度遍历的过程。而且它只返回rdd的直属父shuffle依赖,祖先shuffle依赖不返回,下面有个例子说明:

E <------ A <------ B <------- C
                                |
                     D <--------|
假设上面都是shuffle依赖,getShuffleDependedency(C)只返回B,D

回到方法本身,对C的每一个shuffle依赖B,D,调用getOrCreateShuffleMapStage创建stage。

  1. getOrCreateShuffleMapStage
    假设先对B创建stage,代码如下:
private def getOrCreateShuffleMapStage(
    shuffleDep: ShuffleDependency[_, _, _],
    firstJobId: Int): ShuffleMapStage = {
  shuffleIdToMapStage.get(shuffleDep.shuffleId) match {
    case Some(stage) =>
      stage

    case None =>
      // Create stages for all missing ancestor shuffle dependencies.
      getMissingAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep =>
        // Even though getMissingAncestorShuffleDependencies only returns shuffle dependencies
        // that were not already in shuffleIdToMapStage, it's possible that by the time we
        // get to a particular dependency in the foreach loop, it's been added to
        // shuffleIdToMapStage by the stage creation process for an earlier dependency. See
        // SPARK-13902 for more information.
        if (!shuffleIdToMapStage.contains(dep.shuffleId)) {
          createShuffleMapStage(dep, firstJobId)
        }
      }
      // Finally, create a stage for the given shuffle dependency.
      createShuffleMapStage(shuffleDep, firstJobId)
  }
}
  • 参数shuffleDep,即C到B的依赖,shuffleDep.rdd即为B
  • getMissingAncestorShuffleDependencies(B),此时会返回B所有的祖先shuffle 依赖,也就是 B对A的依赖,和A对E的依赖
  • 下面4中代码中,每个ShuffleMapStage创建后都会映射到shuffle id上,假设新建的ShuffleMapStage作用于B,那么他映射的shuffleid就是B<-C之间的shuffle的id。因此方法里可以根据shuffleId到shuffleIdToMapStage检索,避免重复创建,
  • 对于不存在的ShuffleMapStage, 调用createShuffleMapStage创建stage。
  1. createShuffleMapStage
    对于B而言,先创建A,代码如下:
    def createShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): ShuffleMapStage = {
     val rdd = shuffleDep.rdd
     val numTasks = rdd.partitions.length
     val parents = getOrCreateParentStages(rdd, jobId)
     val id = nextStageId.getAndIncrement()
     val stage = new ShuffleMapStage(id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep)
    
     stageIdToStage(id) = stage
     shuffleIdToMapStage(shuffleDep.shuffleId) = stage
     updateJobIdStageIdMaps(jobId, stage)
    
     if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) {
       // A previously run stage generated partitions for this shuffle, so for each output
       // that's still available, copy information about that output location to the new stage
       // (so we don't unnecessarily re-compute that data).
       val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId)
       val locs = MapOutputTracker.deserializeMapStatuses(serLocs)
       (0 until locs.length).foreach { i =>
         if (locs(i) ne null) {
           // locs(i) will be null if missing
           stage.addOutputLoc(i, locs(i))
         }
       }
     } else {
       // Kind of ugly: need to register RDDs with the cache and map output tracker here
       // since we can't do it in the RDD constructor because # of partitions is unknown
       logInfo("Registering RDD " + rdd.id + " (" + rdd.getCreationSite + ")")
       mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.length)
     }
     stage
    

}

- 此时shuffleDep即B->A的依赖,shuffleDep.rdd就是A
- 上述代码又getOrCreateParentStages(A)创建A的上游,A上游是E,E没有上游节点,此处E所所处ShuffleMapStage被创建,stageid即A->E的shuffleId,stage作用于E
- 所有的上游stages创建完成了,创建当前shuffleDep产生的ShuffleMapStage。建立shuffleId到ShuffleMapStage的映射
- shuffleId在建立RDD DAG图之后就是一直不变的,而stageId每提交一次job都会变动,同一个job可能因为某个环节失败了重新提交,但是失败的job中的某个ShuffleMapStage的输出数据可是完好的可重复利用。if分支使用mapoutTracker检查特定shuffle阶段的数据是不是完好的,然后可以重复利用,避免再此计算。
- 创建stage之后就返回上一层。

下图描述了最终创建出来的stage的依赖图:

#3 提交stage2节最前面,handleJobSubmitted,创建完所有stage之后调用submitStage(finalStage)提交stage,代码如下:

private def submitStage(stage: Stage) {
val jobId = activeJobForStage(stage)
if (jobId.isDefined) {
logDebug("submitStage(" + stage + ")")
if (!waitingStages(stage) && !runningStages(stage) && !failedStages(stage)) {
val missing = getMissingParentStages(stage).sortBy(_.id)
logDebug("missing: " + missing)
if (missing.isEmpty) {
logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents")
submitMissingTasks(stage, jobId.get)
} else {
for (parent <- missing) {
submitStage(parent)
}
waitingStages += stage
}
}
} else {
abortStage(stage, "No active job for stage " + stage.id, None)
}
}

1. waitingStages、runningStage、failedStages分别记录那些还有未完成的上游stage的stage、正在executor上运行的stage和失败的stages
2. getMissingParentStages获得没有提交或者没有完成的上游stage,stage不存在就创建
3. submitMissingTasks在没有未完成的上游stage的情况下,提交当前stage
4. 有上游stage未完成,将当前stage加入到waitingStages队列中

上面submitStages代码中将根据stage创建任务提交的是submitMissingTasks,其代码如下:

private def submitMissingTasks(stage: Stage, jobId: Int) {
// stage可能被多次提交,stage作用的rdd可能在之前几次提交运行中有写partition已经有了计算计算,先找出没有结果的partition
val partitionsToCompute: Seq[Int] = stage.findMissingPartitions()

 ...
// 一个分区一个task,找出一组task运行的位置
val taskIdToLocations: Map[Int, Seq[TaskLocation]] = try {
  stage match {
    case s: ShuffleMapStage =>
      partitionsToCompute.map { id => (id, getPreferredLocs(stage.rdd, id))}.toMap
    case s: ResultStage =>
      partitionsToCompute.map { id =>
        val p = s.partitions(id)
        (id, getPreferredLocs(stage.rdd, p))
      }.toMap
  }
} catch {
    ...
    return
}
 // stage只有一个stageId,但是stage的每次提交都会有新的attempId
stage.makeNewStageAttempt(partitionsToCompute.size, taskIdToLocations.values.toSeq)
listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties))

// 根据stage序列化任务信息,ShuffleMapStage对应ShuffleMapTask,主要信息有task运行作用的rdd,和依赖
// ResultStage对应ResultTask,主要信息有rdd和func(这个func就是前面wordCount代码中foreach 这个action的参数转换来的,只有ResultStage有)
var taskBinary: Broadcast[Array[Byte]] = null
try {

  val taskBinaryBytes: Array[Byte] = stage match {
    case stage: ShuffleMapStage =>
      JavaUtils.bufferToArray(
        closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
    case stage: ResultStage =>
      JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef))
  }
  // 将task信息作为广播变量传输,executor执行task时需要根据广播变量获取task信息,task很多时,使用广播变量传输能有效减少driver上的压力
  taskBinary = sc.broadcast(taskBinaryBytes)
} catch {
  ...
}

// 创建task,一个分区一个task
val tasks: Seq[Task[_]] = try {
val serializedTaskMetrics = closureSerializer.serialize(stage.latestInfo.taskMetrics).array()
stage match {
case stage: ShuffleMapStage =>
stage.pendingPartitions.clear()
partitionsToCompute.map { id =>
val locs = taskIdToLocations(id)
val part = stage.rdd.partitions(id)
stage.pendingPartitions += id
new ShuffleMapTask(stage.id, stage.latestInfo.attemptId,
taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId),
Option(sc.applicationId), sc.applicationAttemptId)
}

    case stage: ResultStage =>
      partitionsToCompute.map { id =>
        val p: Int = stage.partitions(id)
        val part = stage.rdd.partitions(p)
        val locs = taskIdToLocations(id)
        new ResultTask(stage.id, stage.latestInfo.attemptId,
          taskBinary, part, locs, id, properties, serializedTaskMetrics,
          Option(jobId), Option(sc.applicationId), sc.applicationAttemptId)
      }
  }
} catch {
  case NonFatal(e) =>
    abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e))
    runningStages -= stage
    return
}

if (tasks.size > 0) {
  logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " +
    s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})")
  // 使用TaskScheduler提交tasks
  taskScheduler.submitTasks(new TaskSet(
    tasks.toArray, stage.id, stage.latestInfo.attemptId, jobId, properties))
  stage.latestInfo.submissionTime = Some(clock.getTimeMillis())
} else {
  // Because we posted SparkListenerStageSubmitted earlier, we should mark
  // the stage as completed here in case there are no tasks to run
  markStageAsFinished(stage, None)

  val debugString = stage match {
    case stage: ShuffleMapStage =>
      s"Stage ${stage} is actually done; " +
        s"(available: ${stage.isAvailable}," +
        s"available outputs: ${stage.numAvailableOutputs}," +
        s"partitions: ${stage.numPartitions})"
    case stage : ResultStage =>
      s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})"
  }
  logDebug(debugString)
  submitWaitingChildStages(stage)
}

}

上面在代码注释中简单介绍了Stage转换成Task,然后使用TaskScheduler提交的过程。

关于TaskScheduler对任务的管理和提交见另一篇文章[Spark 任务调度-TaskScheduler]()。
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

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

抵扣说明:

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

余额充值