Spark Streaming 生产、消费流程梳理

SparkStreaming流程梳理

根据SparkStreaming的最初设计文档(https://docs.google.com/document/d/1vTCB5qVfyxQPlHuv8rit9-zjdttlgaSrMgfCDQlCJIM/edit#),初版的流程设计如下:

  1. Reciever将block分发至ReceivedBlockHandler;
  2. ReceivedBlockHandler将block保存在内存(无冗余);
  3. Reciever将这个block传输至driver;
  4. Reciever标记该block为recieved;
  5. Driver基于block info信息创建HDFSBackedBlockRDDs;
  6. 基于BlockManagerMaster的block location信息进行调度;
  7. Checkpoint信息存储在HDFS;

而当前稳定版本(2.1.0)的实现中,在多出添加了WAL功能,变更如下:

  1. Reciever将block分发至ReceivedBlockHandler;
  2. ReceivedBlockHandler将block保存在内存(blockManager) + WAL中(无冗余);
  3. Reciever将这个blockInfo传输通过trackerEndpoint 传输至driver;
  4. driver将该blockInfo写入WAL;
  5. Reciever标记该block为recieved;
  6. Driver基于block info信息创建HDFSBackedBlockRDDs(此处也有变更);
  7. 基于BlockManagerMaster的block location信息进行调度;
  8. Checkpoint信息存储在HDFS;

生产阶段

ReceiverSupervisorImpl

ReceiverSupervisorImpl将搜集的内容pushAndReportBlock保存:

 /** Store block and report it to driver */
  def pushAndReportBlock(
      receivedBlock: ReceivedBlock,
      metadataOption: Option[Any],
      blockIdOption: Option[StreamBlockId]
    ) {
    // 构造blockId
    val blockId = blockIdOption.getOrElse(nextBlockId)
    val time = System.currentTimeMillis
    // 调用下述receivedBlockHandler的storeBlock方法,将block保存至blockManager和wal
    val blockStoreResult = receivedBlockHandler.storeBlock(blockId, receivedBlock)
    logDebug(s"Pushed block $blockId in ${(System.currentTimeMillis - time)} ms")
    val numRecords = blockStoreResult.numRecords
    // 根据block信息构造blockInfo
    val blockInfo = ReceivedBlockInfo(streamId, numRecords, metadataOption, blockStoreResult)
    // 传输该blockInfo至driver测的trackerEndpoint
    trackerEndpoint.askWithRetry[Boolean](AddBlock(blockInfo))
    logDebug(s"Reported block $blockId")
  }

receivedBlockHandler

receivedBlockHandler为reciever测的实现wal功能,其主要功能为:将接受到的block并行地保存在blockManger和HDFS中;

  /**
   * This implementation stores the block into the block manager as well as a write ahead log.
   * It does this in parallel, using Scala Futures, and returns only after the block has
   * been stored in both places.
   */
  // 基于Scala Future特质,可以并行地将RecivedBlock存储到blockManager和HDFS
  def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = {

    var numRecords = Option.empty[Long]
    // Serialize the block so that it can be inserted into both
    // 第一步、序列化block
    val serializedBlock = block match {
      case ArrayBufferBlock(arrayBuffer) =>
        numRecords = Some(arrayBuffer.size.toLong)
        serializerManager.dataSerialize(blockId, arrayBuffer.iterator)
      case IteratorBlock(iterator) =>
        val countIterator = new CountingIterator(iterator)
        val serializedBlock = serializerManager.dataSerialize(blockId, countIterator)
        numRecords = countIterator.count
        serializedBlock
      case ByteBufferBlock(byteBuffer) =>
        new ChunkedByteBuffer(byteBuffer.duplicate())
      case _ =>
        throw new Exception(s"Could not push $blockId to block manager, unexpected block type")
    }

    // Store the block in block manager
    // 保存在blockManager的future
    val storeInBlockManagerFuture = Future {
      val putSucceeded = blockManager.putBytes(
        blockId,
        serializedBlock,
        effectiveStorageLevel,
        tellMaster = true)
      if (!putSucceeded) {
        throw new SparkException(
          s"Could not store $blockId to block manager with storage level $storageLevel")
      }
    }

    // Store the block in write ahead log
    // 保存到wal的future
    val storeInWriteAheadLogFuture = Future {
      // 当该函数该write函数完毕,保障该block一定成功地写入hdfs
      writeAheadLog.write(serializedBlock.toByteBuffer, clock.getTimeMillis())
    }

    // Combine the futures, wait for both to complete, and return the write ahead log record handle
    // 参考https://github.com/apache/spark/pull/3721, 该方案使用zip,可以并行地完成上述两者的执行
    val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2)
    val walRecordHandle = ThreadUtils.awaitResult(combinedFuture, blockStoreTimeout)
    WriteAheadLogBasedStoreResult(blockId, numRecords, walRecordHandle)
  }

关于trackerEndpoint

Reciver同Driver之间通过trackerEndpoint通信,其处理上述的AddBlock信息是在ReciverTracker类中实现,其具体实现如下:

 override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
      // Remote messages
      case RegisterReceiver(streamId, typ, host, executorId, receiverEndpoint) =>
        val successful =
          registerReceiver(streamId, typ, host, executorId, receiverEndpoint, context.senderAddress)
        context.reply(successful)
      case AddBlock(receivedBlockInfo) =>
        if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) {
        // 调用receivedBlockTracker.addBlock实现,具体如下
          walBatchingThreadPool.execute(new Runnable {
            override def run(): Unit = Utils.tryLogNonFatalError {
              if (active) {
                context.reply(addBlock(receivedBlockInfo))
              } else {
                throw new IllegalStateException("ReceiverTracker RpcEndpoint shut down.")
              }
            }
          })
        } else {
          context.reply(addBlock(receivedBlockInfo))
        }
      case DeregisterReceiver(streamId, message, error) =>
        deregisterReceiver(streamId, message, error)
        context.reply(true)

      // Local messages
      case AllReceiverIds =>
        context.reply(receiverTrackingInfos.filter(_._2.state != ReceiverState.INACTIVE).keys.toSeq)
      case GetAllReceiverInfo =>
        context.reply(receiverTrackingInfos.toMap)
      case StopAllReceivers =>
        assert(isTrackerStopping || isTrackerStopped)
        stopReceivers()
        context.reply(true)
    }

  /** Add new blocks for the given stream */
  private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = {
    receivedBlockTracker.addBlock(receivedBlockInfo)
  }
  
  /** Add received block. This event will get written to the write ahead log (if enabled). */
  // Driver测处理AddBlock事件
  def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = {
    try {
      // 保存blockInfo信息,writeToLog会判定是否开启wal,
      // 此处要注意: blockInfo信息和在reciever测的block不一样,一个你可以理解为block的meta信息,一个则为真实的数据
      val writeResult = writeToLog(BlockAdditionEvent(receivedBlockInfo))
      if (writeResult) {
        synchronized {
          // 同时将该blockInfo写入blockQueue,供调度使用
          getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo
        }
        logDebug(s"Stream ${receivedBlockInfo.streamId} received " +
          s"block ${receivedBlockInfo.blockStoreResult.blockId}")
      } else {
        logDebug(s"Failed to acknowledge stream ${receivedBlockInfo.streamId} receiving " +
          s"block ${receivedBlockInfo.blockStoreResult.blockId} in the Write Ahead Log.")
      }
      writeResult
    } catch {
      case NonFatal(e) =>
        logError(s"Error adding block $receivedBlockInfo", e)
        false
    }
  }

消费阶段

上述过程为通过reciever进行数据收集的阶段,而产生的block则是通过spark调度任务进行消费的,其消费处理逻辑如下,首先经过JobGenerator每个batchTime生成相应的DStream,然后提交任务,进行处理。

/** Processes all events */
  // JobGenerator启动时,会启动一个定时的timer,根据配置的batchDuration,定时地post GenerateJobs事件,触发生成DStream的逻辑
  private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
    longTime => eventLoop.post(GenerateJobs(new Time(longTime))), "JobGenerator")
    
  private def processEvent(event: JobGeneratorEvent) {
    logDebug("Got event " + event)
    event match {
      // eventLoop收到GenerateJobs事件
      case GenerateJobs(time) => generateJobs(time)
      case ClearMetadata(time) => clearMetadata(time)
      case DoCheckpoint(time, clearCheckpointDataLater) =>
        doCheckpoint(time, clearCheckpointDataLater)
      case ClearCheckpointData(time) => clearCheckpointData(time)
    }
  }
  
  
  //可以看做SparkStreaming的核心调度
  /** Generate jobs and perform checkpointing for the given `time`.  */
  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 {
      // 第一步、分配上述“接受到的block”到该时间点对应的batch;具体实现如下。
      jobScheduler.receiverTracker.allocateBlocksToBatch(time) // allocate received blocks to batch
      // 第二步,等上述分配好allocatedBlocks,调用generateJobs生成Spark定义的Job类(带time参数)
      graph.generateJobs(time) // generate jobs using allocated block
    } match {
      case Success(jobs) =>
        // 第三步、 根据time从inputInfoTracker获取这次time的metaData(这一步没弄明白,为什么不从上述分配好的time->allocatedBlocks开始任务,而要加一个inputInfoTracker),并真正地提交任务,开始计算
        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)
    }
    // 第四步、checkpoint该time至hdfs
    eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = false))
  }
  
  /** Allocate all unallocated blocks to the given batch. */
  // receiverTracker.allocateBlocksToBatch()会调用receivedBlockTracker类
  def allocateBlocksToBatch(batchTime: Time): Unit = {
    if (receiverInputStreams.nonEmpty) {
      receivedBlockTracker.allocateBlocksToBatch(batchTime)
    }
  }
  
 /**
   * Allocate all unallocated blocks to the given batch.
   * This event will get written to the write ahead log (if enabled).
   */
  def allocateBlocksToBatch(batchTime: Time): Unit = synchronized {
    if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) {
      // streamId为Reciever启动时定义的streamId,调用getReceivedBlockQueue().dequeueAll(),将收集到的blockInfo返回,和streamId构成(streamId, blockInfos)的二元组
      val streamIdToBlocks = streamIds.map { streamId =>
          (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true))
      }.toMap
      // 构造成AllocateBlocks对象,方便数据传输
      val allocatedBlocks = AllocatedBlocks(streamIdToBlocks)
      // 在真正的任务开始前,将开始处理做的allocatedBlocks写入wal
      if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) {
        // 如果写入成果,则开始分配任务,在time->allocatedBlocks添加该相应对,等待generateJob()使用
        timeToAllocatedBlocks.put(batchTime, allocatedBlocks)
        lastAllocatedBatchTime = batchTime
      } else {
        // 如果写入wal失败,则需要重试
        logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery")
      }
    } else {
      // This situation occurs when:
      // 1. WAL is ended with BatchAllocationEvent, but without BatchCleanupEvent,
      // possibly processed batch job or half-processed batch job need to be processed again,
      // so the batchTime will be equal to lastAllocatedBatchTime.
      // 2. Slow checkpointing makes recovered batch time older than WAL recovered
      // lastAllocatedBatchTime.
      // This situation will only occurs in recovery time.
      logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery")
    }
  }

GenerateJob如何生成RDD?

从Spark Streaming的定义来讲,大家都熟悉Spark Streaming是一个批处理,将流转换成离散的DStream。
但这个过程却十分复杂,具体可以参考这个链接:https://github.com/lw-lin/CoolplaySpark/blob/master/Spark%20Streaming%20源码解析系列/1.2%20DStream%20生成%20RDD%20实例详解.md

后续

SparkStreaming的容错机制有点绕,名字都叫wal,其实含义有些不同,后面会有一篇文章介绍其wal容错机制,可以参考https://databricks.com/blog/2015/01/15/improved-driver-fault-tolerance-and-zero-data-loss-in-spark-streaming.html这篇文章,讲解的挺详细的;

参考:

  • ReceivedBlockTracker WAL实现: https://issues.apache.org/jira/browse/SPARK-7139
  • Databrick 关于一致性的文章: https://databricks.com/blog/2015/01/15/improved-driver-fault-tolerance-and-zero-data-loss-in-spark-streaming.html
  • Spark Streaming WAL实现:https://issues.apache.org/jira/browse/SPARK-3129
  • Spark Streaming HA 设计: https://docs.google.com/document/d/1vTCB5qVfyxQPlHuv8rit9-zjdttlgaSrMgfCDQlCJIM/edit#
  • https://github.com/apache/spark/pull/3721
  • https://github.com/lw-lin/CoolplaySpark/blob/master/Spark%20Streaming%20源码解析系列/1.2%20DStream%20生成%20RDD%20实例详解.md

你可能感兴趣的:(Spark Streaming 生产、消费流程梳理)