前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >[spark streaming] 动态生成 Job 并提交执行

[spark streaming] 动态生成 Job 并提交执行

作者头像
UFO
发布2018-08-29 17:38:56
1.2K0
发布2018-08-29 17:38:56
举报
文章被收录于专栏:Spark生态圈Spark生态圈

前言

Spark Streaming Job的生成是通过JobGenerator每隔 batchDuration 长时间动态生成的,每个batch 对应提交一个JobSet,因为针对一个batch可能有多个输出操作。

概述流程:

  • 定时器定时向 eventLoop 发送生成job的请求
  • 通过receiverTracker 为当前batch分配block
  • 为当前batch生成对应的 Jobs
  • 将Jobs封装成JobSet 提交执行

入口

在 JobGenerator 初始化的时候就创建了一个定时器:

代码语言:javascript
复制
private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
    longTime => eventLoop.post(GenerateJobs(new Time(longTime))), "JobGenerator")

每隔 batchDuration 就会向 eventLoop 发送 GenerateJobs(new Time(longTime))消息,eventLoop的事件处理方法中会调用generateJobs(time)方法:

代码语言:javascript
复制
      case GenerateJobs(time) => generateJobs(time)
代码语言:javascript
复制
private def generateJobs(time: Time) {
    // Checkpoint all RDDs marked for checkpointing to ensure their lineages are
    // truncated periodically. Otherwise, we may run into stack overflows (SPARK-6847).
    ssc.sparkContext.setLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS, "true")
    Try {
      jobScheduler.receiverTracker.allocateBlocksToBatch(time) // allocate received blocks to batch
      graph.generateJobs(time) // generate jobs using allocated block
    } match {
      case Success(jobs) =>
        val streamIdToInputInfos = jobScheduler.inputInfoTracker.getInfo(time)
        jobScheduler.submitJobSet(JobSet(time, jobs, streamIdToInputInfos))
      case Failure(e) =>
        jobScheduler.reportError("Error generating jobs for time " + time, e)
        PythonDStream.stopStreamingContextIfPythonProcessIsDead(e)
    }
    eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = false))
  }

为当前batchTime分配Block

首先调用receiverTracker.allocateBlocksToBatch(time)方法为当前batchTime分配对应的Block,最终会调用receiverTracker的Block管理者receivedBlockTrackerallocateBlocksToBatch方法:

代码语言:javascript
复制
def allocateBlocksToBatch(batchTime: Time): Unit = synchronized {
    if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) {
      val streamIdToBlocks = streamIds.map { streamId =>
          (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true))
      }.toMap
      val allocatedBlocks = AllocatedBlocks(streamIdToBlocks)
      if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) {
        timeToAllocatedBlocks.put(batchTime, allocatedBlocks)
        lastAllocatedBatchTime = batchTime
      } else {
        logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery")
      }
    } else {
      logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery")
    }
  }
代码语言:javascript
复制
private def getReceivedBlockQueue(streamId: Int): ReceivedBlockQueue = {
    streamIdToUnallocatedBlockQueues.getOrElseUpdate(streamId, new ReceivedBlockQueue)
  }

可以看到是从streamIdToUnallocatedBlockQueues中获取到所有streamId对应的未分配的blocks,该队列的信息是supervisor 存储好Block后向receiverTracker上报的Block信息,详情可见 ReceiverTracker 数据产生与存储

获取到所有streamId对应的未分配的blockInfos后,将其放入了timeToAllocatedBlocks:Map[Time, AllocatedBlocks]中,后面生成RDD的时候会用到。

为当前batchTime生成Jobs

调用DStreamGraphgenerateJobs方法为当前batchTime生成job:

代码语言:javascript
复制
 def generateJobs(time: Time): Seq[Job] = {
    logDebug("Generating jobs for time " + time)
    val jobs = this.synchronized {
      outputStreams.flatMap { outputStream =>
        val jobOption = outputStream.generateJob(time)
        jobOption.foreach(_.setCallSite(outputStream.creationSite))
        jobOption
      }
    }
    logDebug("Generated " + jobs.length + " jobs for time " + time)
    jobs
  }

一个outputStream就对应一个job,遍历所有的outputStreams,为其生成job:

代码语言:javascript
复制
# ForEachDStream
override def generateJob(time: Time): Option[Job] = {
    parent.getOrCompute(time) match {
      case Some(rdd) =>
        val jobFunc = () => createRDDWithLocalProperties(time, displayInnerRDDOps) {
          foreachFunc(rdd, time)
        }
        Some(new Job(time, jobFunc))
      case None => None
    }
  }

先获取到time对应的RDD,然后将其作为参数再调用foreachFunc方法,foreachFunc方法是通过构造器传过来的,我们来看看print()输出的情况:

代码语言:javascript
复制
def print(num: Int): Unit = ssc.withScope {
    def foreachFunc: (RDD[T], Time) => Unit = {
      (rdd: RDD[T], time: Time) => {
        val firstNum = rdd.take(num + 1)
        // scalastyle:off println
        println("-------------------------------------------")
        println(s"Time: $time")
        println("-------------------------------------------")
        firstNum.take(num).foreach(println)
        if (firstNum.length > num) println("...")
        println()
        // scalastyle:on println
      }
    }
    foreachRDD(context.sparkContext.clean(foreachFunc), displayInnerRDDOps = false)
  }

这里的构造的foreachFunc方法就是最终和rdd一起提交job的执行方法,也即对rdd调用take()后并打印,真正触发action操作的是在这个func函数里,现在再来看看是怎么拿到rdd的,每个DStream都有一个generatedRDDs:Map[Time, RDD[T]]变量,来保存time对应的RDD,若获取不到则会通过compute()方法来计算,对于需要在executor上启动Receiver来接收数据的ReceiverInputDStream来说:

代码语言:javascript
复制
 override def compute(validTime: Time): Option[RDD[T]] = {
    val blockRDD = {

      if (validTime < graph.startTime) {
        // If this is called for any time before the start time of the context,
        // then this returns an empty RDD. This may happen when recovering from a
        // driver failure without any write ahead log to recover pre-failure data.
        new BlockRDD[T](ssc.sc, Array.empty)
      } else {
        // Otherwise, ask the tracker for all the blocks that have been allocated to this stream
        // for this batch
        val receiverTracker = ssc.scheduler.receiverTracker
        val blockInfos = receiverTracker.getBlocksOfBatch(validTime).getOrElse(id, Seq.empty)

        // Register the input blocks information into InputInfoTracker
        val inputInfo = StreamInputInfo(id, blockInfos.flatMap(_.numRecords).sum)
        ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo)

        // Create the BlockRDD
        createBlockRDD(validTime, blockInfos)
      }
    }
    Some(blockRDD)
  }

会通过receiverTracker来获取该batch对应的blocks,前面已经分析过为所有streamId分配了对应的未分配的block,并且放在了timeToAllocatedBlocks:Map[Time, AllocatedBlocks]中,这里底层就是从这个timeToAllocatedBlocks获取到的blocksInfo,然后调用了createBlockRDD(validTime, blockInfos)通过blockId创建了RDD。

最后,将通过此RDD和foreachFun构建jobFunc,并创建Job返回。

封装jobs成JobSet并提交执行

每个outputStream对应一个Job,最终就会生成一个jobs,为这个jobs创建JobSet,并通过jobScheduler.submitJobSet(JobSet(time, jobs, streamIdToInputInfos))来提交这个JobSet:

代码语言:javascript
复制
jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job)))

然后通过jobExecutor来执行,jobExecutor是一个线程池,并行度默认为1,可通过spark.streaming.concurrentJobs配置,即同时可执行几个批次的数据。

处理类JobHandler中调用的是Job.run(),执行的是前面构建的 jobFunc 方法。

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

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

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

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

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
目录
  • 前言
  • 入口
  • 为当前batchTime分配Block
  • 为当前batchTime生成Jobs
  • 封装jobs成JobSet并提交执行
领券
问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档