前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >sparkstreaming的状态计算-updateStateByKey源码

sparkstreaming的状态计算-updateStateByKey源码

作者头像
sanmutongzi
发布2020-03-04 19:20:10
3540
发布2020-03-04 19:20:10
举报
文章被收录于专栏:stream processstream process

转发请注明原创地址:https://cloud.tencent.com/developer/article/1594182

本文基于spark源码版本为2.4.3

在流式计算中通常会有状态计算的需求,即当前计算结果不仅依赖于目前收到数据还需要之前结果进行合并计算的场景,由于sparkstreaming的mini-batch机制,必须将之前的状态结果存储在RDD中并在下一次batch计算时将其取出进行合并,这就是updateStateByKey方法的用处。

简单用例:

  def main(args: Array[String]): Unit = {

    val host = "localhost"
    val port = "8001"

    StreamingExamples.setStreamingLogLevels()

    // Create the context with a 1 second batch size
    val sparkConf = new SparkConf().setMaster("local[4]").setAppName("NetworkWordCount")
    val ssc = new StreamingContext(sparkConf, Seconds(10))


    ssc.checkpoint("/Users/dyang/Desktop/checkpoittmp")

    val lines = ssc.socketTextStream(host, port.toInt, StorageLevel.MEMORY_AND_DISK_SER)
    val words = lines.flatMap(_.split(" "))
    val wordCounts: DStream[(String, Int)] = words.map(x => (x, 1))
   //.reduceByKey(_ + _)

    val totalCounts = wordCounts.updateStateByKey{(values:Seq[Int],state:Option[Int])=>   Some(values.sum + state.getOrElse(0))}
    totalCounts.print()
    ssc.start()
    ssc.awaitTermination()
  }

  上面例子展示了一个简单的wordcount版本的有状态统计,在updateStateByKey的作用下,应用会记住每个word之前count的总和并把下次到来的数据进行累加.

updateStateByKey拥有不同的参数封装版本,比较全的一个定义如下

  /**
   * Return a new "state" DStream where the state for each key is updated by applying
   * the given function on the previous state of the key and the new values of each key.
   * In every batch the updateFunc will be called for each state even if there are no new values.
   * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
   * @param updateFunc State update function. Note, that this function may generate a different
   *                   tuple with a different key than the input key. Therefore keys may be removed
   *                   or added in this way. It is up to the developer to decide whether to
   *                   remember the partitioner despite the key being changed.
   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new
   *                    DStream
   * @param rememberPartitioner Whether to remember the partitioner object in the generated RDDs.
   * @tparam S State type
   */
  def updateStateByKey[S: ClassTag](
      updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
      partitioner: Partitioner,
      rememberPartitioner: Boolean): DStream[(K, S)] = ssc.withScope {
    val cleanedFunc = ssc.sc.clean(updateFunc)
    val newUpdateFunc = (_: Time, it: Iterator[(K, Seq[V], Option[S])]) => {
      cleanedFunc(it)
    }
    new StateDStream(self, newUpdateFunc, partitioner, rememberPartitioner, None)
  }

  其中,参数里的updateFunc的是用户原本传入函数updateFunc: (SeqV, OptionS) => OptionS的一次转化:

    val cleanedUpdateF: (Seq[V], Option[S]) => Option[S] = sparkContext.clean(updateFunc)
    val newUpdateFunc = (iterator: Iterator[(K, Seq[V], Option[S])]) => {
      iterator.flatMap(t => {
        cleanedUpdateF(t._2, t._3).map(s => (t._1, s))
      })
    }
    updateStateByKey(newUpdateFunc, partitioner, true)

  最终updateStateByKey的结果是将一个PairDStreamFunctions转化成了一个StateDStream。对于所有的Dstream,compute(time)方法都是他们生成每个duration RDD的具体实现

  override def compute(validTime: Time): Option[RDD[(K, S)]] = {

    // Try to get the previous state RDD
    getOrCompute(validTime - slideDuration) match {

      case Some(prevStateRDD) =>    // If previous state RDD exists
        // Try to get the parent RDD
        parent.getOrCompute(validTime) match {
          case Some(parentRDD) =>    // If parent RDD exists, then compute as usual
            computeUsingPreviousRDD (validTime, parentRDD, prevStateRDD)
          case None =>     // If parent RDD does not exist
            // Re-apply the update function to the old state RDD
            val updateFuncLocal = updateFunc
            val finalFunc = (iterator: Iterator[(K, S)]) => {
              val i = iterator.map(t => (t._1, Seq.empty[V], Option(t._2)))
              updateFuncLocal(validTime, i)
            }
            val stateRDD = prevStateRDD.mapPartitions(finalFunc, preservePartitioning)
            Some(stateRDD)
        }

      case None =>    // If previous session RDD does not exist (first input data)
        // Try to get the parent RDD
        parent.getOrCompute(validTime) match {
          case Some(parentRDD) =>   // If parent RDD exists, then compute as usual
            initialRDD match {
              case None =>
                // Define the function for the mapPartition operation on grouped RDD;
                // first map the grouped tuple to tuples of required type,
                // and then apply the update function
                val updateFuncLocal = updateFunc
                val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => {
                  updateFuncLocal (validTime,
                    iterator.map (tuple => (tuple._1, tuple._2.toSeq, None)))
                }

                val groupedRDD = parentRDD.groupByKey(partitioner)
                val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning)
                // logDebug("Generating state RDD for time " + validTime + " (first)")
                Some (sessionRDD)
              case Some (initialStateRDD) =>
                computeUsingPreviousRDD(validTime, parentRDD, initialStateRDD)
            }
          case None => // If parent RDD does not exist, then nothing to do!
            // logDebug("Not generating state RDD (no previous state, no parent)")
            None
        }
    }
  }

这里需要解释一下parent的含义:parent,是本 DStream 上游依赖的 DStream,从上面updateStateByKey最后对StateDstream实例化代码可知,它将self也就是生成PairDStreamFunctions的Dstream本身传了进来构造了Dstream之间的DAG关系。

每个Dstream内部通过一个HashMapTime, RDD[T]来管理已经生成过的RDD列表, key 是一个 Time;这个 Time 是与用户指定的 batchDuration 对齐了的时间 —— 如每 15s 生成一个 batch 的话,那么这里的 key 的时间就是 08h:00m:00s08h:00m:15s 这种,所以其实也就代表是第几个 batch。generatedRDD 的 value 就是 RDD 的实例,所以parent.getOrCompute(validTime)这个调用表示了获取经过上游Dstream的transfer操作后生成对应的RDD。

上述源码已经带了非常详细的注释,排除掉各种parentRDD/(prevStateRDD/initialRDD)不完整的边界情况之后,方法进入到了合并当前数据和历史状态的方法:computeUsingPreviousRDD

  private [this] def computeUsingPreviousRDD(
      batchTime: Time,
      parentRDD: RDD[(K, V)],
      prevStateRDD: RDD[(K, S)]) = {
    // Define the function for the mapPartition operation on cogrouped RDD;
    // first map the cogrouped tuple to tuples of required type,
    // and then apply the update function
    val updateFuncLocal = updateFunc
    val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))]) => {
      val i = iterator.map { t =>
        val itr = t._2._2.iterator
        val headOption = if (itr.hasNext) Some(itr.next()) else None
        (t._1, t._2._1.toSeq, headOption)
      }
      updateFuncLocal(batchTime, i)
    }
    val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner)
    val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning)
    Some(stateRDD)
  }

这个方法首先将当前数据parentRDD和prevStateRDD进行了cogroup运算,返回的数据类型位RDD[(K, (IterableV, IterableS))],其中K是DStream的key的类型,value类型是当前数据的terableV和历史状态的IterableS)的二元Tuple,为了匹配这个参数类型spark将前面的updateFunc: (Iterator[(K, SeqV, OptionS)])继续进行了封装

   val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))])

反过来看就是,最初形式为(K, (IterableV, IterableS))的RDD数据经过一次封装变成了(Iterator[(K, SeqV, OptionS)]格式再经过第二次封装变成了对用户自定义状态函数updateFunc: (SeqV, OptionS) => OptionS的调用并返回RDD(K, S)格式的RDD。

注:

1 在spark源码中存在大量的隐式转换,比如updateStateByKey方法并不存在Dstream而是PairDStreamFunctions对象内,这是由于DStream的伴生对象中有一个隐式转换

  implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)])
      (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null):
    PairDStreamFunctions[K, V] = {
    new PairDStreamFunctions[K, V](stream)
  }

  所有符合DStream(K, V)类型的key-value都会通过这个隐式转换适配成PairDStreamFunctions对象

2 在使用状态算子的时候必须打开checkpoint功能,程序启动器就无法通过条件检查报错:

java.lang.IllegalArgumentException: requirement failed: The checkpoint directory has not been set. Please set it by StreamingContext.checkpoint()

参考文献:

1 DStream 生成 RDD 实例详解

2 Spark源码中隐式转换的使用

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

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

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

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

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
相关产品与服务
批量计算
批量计算(BatchCompute,Batch)是为有大数据计算业务的企业、科研单位等提供高性价比且易用的计算服务。批量计算 Batch 可以根据用户提供的批处理规模,智能地管理作业和调动其所需的最佳资源。有了 Batch 的帮助,您可以将精力集中在如何分析和处理数据结果上。
领券
问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档