前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >Spark 行动算子源码分析

Spark 行动算子源码分析

作者头像
Tim在路上
发布2022-03-23 14:16:01
2160
发布2022-03-23 14:16:01
举报

Spark 行动算子源码分析

action算子都是直接调用sc.runJob(this, func _), 在调用时将func传给分区执行,并在调用后,在Driver端对数据在执行自定义的函数。

  • count 算子

返回RDD中的元素个数。

代码语言:javascript
复制
def count(): Long = sc.runJob(this, Utils.getIteratorSize_).sum

def getIteratorSize(iterator: Iterator[_]): Long = {
    var count = 0L
    while (iterator.hasNext) {
      count += 1L
      iterator.next()
    }
    count
  }

可以从runJob的源码实现可以看出count函数是给每一个分区传入了遍历统计的函数,在执行runJob后,将每一个分区元素个数封装为Array进行返回,最后执行一个sum,统计整个的RDD的元素个数。

代码语言:javascript
复制
def runJob[T, U: ClassTag](
    rdd: RDD[T],
    func: Iterator[T] => U,
    partitions: Seq[Int]): Array[U] = {
  val cleanedFunc = clean(func)
  runJob(rdd, (ctx: TaskContext, it: Iterator[T]) => cleanedFunc(it), partitions)
}

runJob中clean函数的作用就是递归清理外围类中无用域,降低序列化的开销,防止不必要的不可序列化异常。之后我们会详细的介绍clean的函数。

代码语言:javascript
复制
def runJob[T, U: ClassTag](
    rdd: RDD[T],
    func: (TaskContext, Iterator[T]) => U,
    partitions: Seq[Int],
    resultHandler: (Int, U) => Unit): Unit = {
  // 判断用户是否stop
  if (stopped.get()) {
    throw new IllegalStateException("SparkContext has been shutdown")
  }
  val callSite = getCallSite
  val cleanedFunc = clean(func)
  logInfo("Starting job: " + callSite.shortForm)
  if (conf.getBoolean("spark.logLineage", false)) {
    logInfo("RDD's recursive dependencies:\n" + rdd.toDebugString)
  }
  // 调用dagScheduler的runJob
  dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, resultHandler,localProperties.get)
  // 标记进度条已完成
  progressBar.foreach(_.finishAll())
  // 调用checkpoint
  rdd.doCheckpoint()
}

通过源码可以发现,action算子会生成一个job, 并将该job提交给dagScheduler进行执行。执行完成后在调用checkpoint(), 它会根据依赖依次执行每一个RDD的checkpoint, 只有定义了checkpointData, 才会真正执行。

代码语言:javascript
复制
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)
  }
  // 自增形式获取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)
  // 函数转换为Task 迭代器类型
  val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
  val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
// 将任务异步执行,提交到阻塞队列待线程调用提交任务
eventProcessLoop.post(JobSubmitted(
    jobId, rdd, func2, partitions.toArray, callSite, waiter,
    SerializationUtils.clone(properties)))
  waiter
}
代码语言:javascript
复制
override def run(): Unit = {
  try {
    while (!stopped.get) {
      val event =eventQueue.take()
      try {
        // 从阻塞队列中取出JobSubmitted实际
        onReceive(event)
      } catch {
        caseNonFatal(e) =>
          try {
            onError(e)
          } catch {
            caseNonFatal(e) => logError("Unexpected error in " + name, e)
          }
      }
    }
  }
...
private def doOnReceive(event: DAGSchedulerEvent): Unit = event match {
    case JobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) =>
      // 调用handleJobSubmitted方法
      dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties)
...
代码语言:javascript
复制
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.
    // 逆序按照shuffle进行切分stage, 返回最后一个stage
    finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite)
  } catch {
      ...
  }
  // Job submitted, clear internal data.
barrierJobIdToNumTasksCheckFailures.remove(jobId)
  // 封装activeJob
  val job = new ActiveJob(jobId, finalStage, callSite, listener, properties)
  clearCacheLocs()
  logInfo("Got job %s (%s) with %d output partitions".format(
    job.jobId, callSite.shortForm, partitions.length))
  logInfo("Final stage: " + finalStage + " (" + finalStage.name+ ")")
  logInfo("Parents of final stage: " + finalStage.parents)
  logInfo("Missing parents: " + getMissingParentStages(finalStage))

  val jobSubmissionTime = clock.getTimeMillis()
jobIdToActiveJob(jobId) = job
activeJobs+= job
  finalStage.setActiveJob(job)
  // 绑定job和stage
  val stageIds =jobIdToStageIds(jobId).toArray
  val stageInfos = stageIds.flatMap(id =>stageIdToStage.get(id).map(_.latestInfo))
 // 监听job
  listenerBus.post(
SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties))
 // 提交stage
  submitStage(finalStage)
}

总的来说,spark任务在action算子时,会提交一个job, 并将job提交给dagScheduler, dagScheduler 将其封装为JobSubmitted对象,以异步的形式提交,线程拿到JobSubmitted获得其finalStage并判断其为resultStage或ShuffleMapStage, (前者有返回,后者无返回),再逆序的根据宽窄依赖将其划分为不同的stage, 最后将每一个stage,按照分区拆分为Tasksets, 最终提交给TaskManage,待Executor资源准备好后进行申请Task。

  • reduce 算子

使用关联和合并的方式减少RDD中的元素。

代码语言:javascript
复制
def reduce(f: (T, T) => T): T = withScope {
  val cleanF = sc.clean(f)
  val reducePartition: Iterator[T] => Option[T] = iter => {
    if (iter.hasNext) {
      Some(iter.reduceLeft(cleanF))
    } else {
      None
    }
  }
  var jobResult: Option[T] = None
  val mergeResult = (index: Int, taskResult: Option[T]) => {
    if (taskResult.isDefined) {
      jobResult = jobResult match {
        case Some(value) => Some(f(value, taskResult.get))
        case None => taskResult
      }
    }
  }
  sc.runJob(this, reducePartition, mergeResult)
  // Get the final result out of our Option, or throw an exception if the RDD was empty
  jobResult.getOrElse(throw new UnsupportedOperationException("empty collection"))
}

reduce算子可以看出,其定义了reducePartition在每一个分区执行的,即reduceLeft, 同时定义了一个mergeResult用于回收合并元素。mergeResult函数是作为resultHandler传入的,这不同于将结果回收到driver后再进行处理。

代码语言:javascript
复制
override def taskSucceeded(index: Int, result: Any): Unit = {
  // resultHandler call must be synchronized in case resultHandler itself is not thread safe.
  synchronized {
    resultHandler(index, result.asInstanceOf[T])
  }
  if (finishedTasks.incrementAndGet() == totalTasks) {
jobPromise.success(())
  }
}

resultHandler是在任务成功后以同步的形式进行调用。

  • collect 算子

返回包含所有元素的数组。

代码语言:javascript
复制
def collect(): Array[T] = withScope {
  val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
  Array.concat(results: _*)
}
代码语言:javascript
复制
def concat[T: ClassTag](xss: Array[T]*): Array[T] = {
  val b =newBuilder[T]
  b.sizeHint(xss.map(_.length).sum)
  for (xs <- xss) b ++= xs
  b.result()
}

从源码可以看出collect是将分区迭代器转换为Array, 返回driver后在将其统一回收到一个数组中。

  • take 算子

取RDD中前num个元素,其工作原理为首先扫描一个分区,根据该分区的结果来估计还需要扫描分区的个数。

代码语言:javascript
复制
def take(num: Int): Array[T] = withScope {
  val scaleUpFactor = Math.max(conf.getInt("spark.rdd.limit.scaleUpFactor", 4), 2)
  if (num == 0) {
    new Array[T](0)
  } else {
    val buf = new ArrayBuffer[T]
    val totalParts = this.partitions.length
    var partsScanned = 0
    while (buf.size < num && partsScanned < totalParts) {
      // The number of partitions to try in this iteration. It is ok for this number to be
      // greater than totalParts because we actually cap it at totalParts in runJob.
      var numPartsToTry = 1L
      val left = num - buf.size
      if (partsScanned > 0) {
        // If we didn't find any rows after the previous iteration, quadruple and retry.
        // Otherwise, interpolate the number of partitions we need to try, but overestimate
        // it by 50%. We also cap the estimation in the end.
        if (buf.isEmpty) {
          numPartsToTry = partsScanned * scaleUpFactor
        } else {
          // As left > 0, numPartsToTry is always >= 1
          numPartsToTry = Math.ceil(1.5 * left * partsScanned / buf.size).toInt
          numPartsToTry = Math.min(numPartsToTry, partsScanned * scaleUpFactor)
        }
      }

      val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt)
      val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p)

      res.foreach(buf ++= _.take(num - buf.size))
      partsScanned += p.size
    }

    buf.toArray
  }
}
  • takeOrdered 算子

返回按照指定顺序排序的最小num个元素。

代码语言:javascript
复制
def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = withScope {
  if (num == 0) {
    Array.empty
} else {
    // 在每个分区上创建最小堆,
    val mapRDDs = mapPartitions { items =>
      // Priority keeps the largest elements, so let's reverse the ordering.
      val queue = new BoundedPriorityQueue[T](num)(ord.reverse)
      queue ++= collectionUtils.takeOrdered(items, num)(ord)
Iterator.single(queue)
    }
    if (mapRDDs.partitions.length == 0) {
      Array.empty
} else {
      mapRDDs.reduce { (queue1, queue2) =>
        queue1 ++= queue2
        queue1
      }.toArray.sorted(ord)
    }
  }
}

在每个分区上创建容量为num的最小堆,获取分区上的最小num个元素。然后调用reduce, 将每个分区返回的queue进行合并为num的最小堆。top的实现就是调用了takeOrdered只是排序的顺序相反。

  • lookup 算子

查看传入key对应的value的值,返回是个数组

代码语言:javascript
复制
def lookup(key: K): Seq[V] = self.withScope {
  self.partitioner match {
    case Some(p) =>
      val index = p.getPartition(key)
      val process = (it: Iterator[(K, V)]) => {
        val buf = new ArrayBuffer[V]
        for (pair <- it if pair._1 == key) {
          buf += pair._2
        }
        buf
      } : Seq[V]
      val res = self.context.runJob(self, process,Array(index))
      res(0)
    case None =>
      self.filter(_._1 == key).map(_._2).collect()
  }
}

如果存在分区器,则通过key获取其所在的分区id, 调用runJob获取对应分区并和key相同的元素的value. 否则就通过filter和map,进行collect获取对应的值。

  • aggregate 算子

聚合分区内的元素,回收分区聚合结果,并将其应用于合并函数。

代码语言:javascript
复制
def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = withScope {
  // Clone the zero value since we will also be serializing it as part of tasks
  var jobResult = Utils.clone(zeroValue, sc.env.serializer.newInstance())
  val cleanSeqOp = sc.clean(seqOp)
  val cleanCombOp = sc.clean(combOp)
  val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp)
  val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult)
  sc.runJob(this, aggregatePartition, mergeResult)
  jobResult
}

aggregate的实现原理和上文基本一样,定义aggregatePartition传送给分区进行分区内的聚合, mergeResult作为resultHandler在分区执行成功后进行同步执行。

本文参与 腾讯云自媒体分享计划,分享自作者个人站点/博客。
原始发表:2022.02.08 ,如有侵权请联系 cloudcommunity@tencent.com 删除

本文分享自 作者个人站点/博客 前往查看

如有侵权,请联系 cloudcommunity@tencent.com 删除。

本文参与 腾讯云自媒体分享计划  ,欢迎热爱写作的你一起参与!

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
目录
  • Spark 行动算子源码分析
相关产品与服务
文件存储
文件存储(Cloud File Storage,CFS)为您提供安全可靠、可扩展的共享文件存储服务。文件存储可与腾讯云服务器、容器服务、批量计算等服务搭配使用,为多个计算节点提供容量和性能可弹性扩展的高性能共享存储。腾讯云文件存储的管理界面简单、易使用,可实现对现有应用的无缝集成;按实际用量付费,为您节约成本,简化 IT 运维工作。
领券
问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档