为了账号安全,请及时绑定邮箱和手机立即绑定

Spark Streaming源码解读之State管理之updateStateByKey和mapWithState解密

标签:
Spark

从这节课开始,简介Spark Streaming的状态管理。

SparkStreaming 是按BatchDuration来划分Job的,但我们有时需要根据业务要求按照另外的时间周期(比如说,对过去24小时、或者过去一周的数据,等等这些大于BatchDuration的周期),对数据进行处理(比如计算最近24小时的销售额排名、今年的最新销售量等)。这需要根据之前的计算结果和新时间周期的数据,计算出新的计算结果。

updateStateByKey和mapWithState都是针对类型的数据进行操作,而RDD类本身并不对 类型的数据进行操作,所以要借助隐式转换。隐式转换放在了DStream伴生对象的区域。

object DStream {

// `toPairDStreamFunctions` was in SparkContext before 1.3 and users had to

// `import StreamingContext._` to enable it. Now we move it here to make the compiler find

// it automatically. However, we still keep the old function in StreamingContext for backward

// compatibility and forward to the following function directly.

implicit deftoPairDStreamFunctions[K, V](stream: DStream[(K, V)])

(implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null):

PairDStreamFunctions[K, V] = {

newPairDStreamFunctions[K, V](stream)

}

...

}

生成了PairDStreamFunctions对象。PairDStreamFunctions类中有updateStateByKey、mapWithState这些功能。


1.updateStateByKey解密

先看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.

* Hash partitioning is used to generate the RDDs with Spark's default number of partitions.

* @param updateFunc State update function. If `this` function returns None, then

*                   corresponding state key-value pair will be eliminated.

* @tparam S State type

*/

defupdateStateByKey[S: ClassTag](

updateFunc: (Seq[V], Option[S]) => Option[S]

): DStream[(K, S)] = ssc.withScope {

updateStateByKey(updateFunc,defaultPartitioner())

}

updateStateByKey返回的都是DStream类型。

根据updateFunc这个函数来更新状态。其中参数:Seq[V]是本次的数据类型,Option[S]是前次计算结果类型,本次计算结果类型也是Option[S]。

计算肯定需要Partitioner。因为Hash高效率且不做排序,默认Partitioner是HashPartitoner。

PairDStreamFunction.defaultPartitioner:

private[streaming] defdefaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = {

newHashPartitioner(numPartitions)

}

看其中返回值类型为StateDStream的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.

* 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 paritioner object in the generated RDDs.

* @tparam S State type

*/

defupdateStateByKey[S: ClassTag](

updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],

partitioner: Partitioner,

rememberPartitioner: Boolean

): DStream[(K, S)] = ssc.withScope {

newStateDStream(self, ssc.sc.clean(updateFunc), partitioner, rememberPartitioner, None)

}

看看这个StateDStream:

classStateDStream[K: ClassTag, V: ClassTag, S: ClassTag](

parent: DStream[(K, V)],

updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],

partitioner: Partitioner,

preservePartitioning: Boolean,

initialRDD : Option[RDD[(K, S)]]

) extends DStream[(K, S)](parent.ssc) {

super.persist(StorageLevel.MEMORY_ONLY_SER)

...

是基于内存的。

所有像StateDStream这样的DStream子类都要覆写compute方法。

StateDStream.compute:

override defcompute(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(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[V](), Option(t._2)))

updateFuncLocal(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 (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(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

}

}

}

}

}

其中会用到computeUsingPreviousRDD方法。去看看。

StateDStream.computeUsingPreviousRDD:

private [this] defcomputeUsingPreviousRDD(

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(i)

}

val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner)

val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning)

Some(stateRDD)

}

由于cogroup会对所有数据进行扫描,再按key进行分组,所以性能上会有问题。特别是随着时间的推移,这样的计算到后面会越算越慢。

所以数据量大的计算、复杂的计算,都不建议使用updateStateByKey。

2.mapWithState解密

虽然有人使用mapWithState后感觉效果还可以,但源码中仍表明,mapWithState还在试验状态。

mapWithState方法有多个。先看第一个。


PairDStreamFunctions.mapWithState:

/**

* :: Experimental ::

* Return a [[MapWithStateDStream]] by applying a function to every key-value element of

* `this` stream, while maintaining some state data for each unique key. The mapping function

* and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this

* transformation can be specified using [[StateSpec]] class. The state data is accessible in

* as a parameter of type [[State]] in the mapping function.

*

* Example of using `mapWithState`:

* {{{

*    // A mapping function that maintains an integer state and return a String

*    def mappingFunction(key: String, value: Option[Int], state:State[Int]): Option[String] = {

*      // Use state.exists(), state.get(), state.update() and state.remove()

*      // to manage state, and return the necessary string

*    }

*

*    val spec = StateSpec.function(mappingFunction).numPartitions(10)

*

*    val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec)

* }}}

*

* @param spec          Specification of this transformation

* @tparam StateType    Class type of the state data

* @tparam MappedType   Class type of the mapped data

*/

@Experimental

defmapWithState[StateType: ClassTag, MappedType: ClassTag](

spec:StateSpec[K, V, StateType, MappedType]

):MapWithStateDStream[K, V, StateType, MappedType] = {

new MapWithStateDStreamImpl[K, V, StateType, MappedType](

self,

spec.asInstanceOf[StateSpecImpl[K, V, StateType, MappedType]]

)

}

注释中给出了一个mapWithState使用实例。先要定义一个mappingFunction。mappingFunction的参数中,State类型的state是历史数据,相当于一个内存数据表;key指明是对state中的哪个键进行操作;value指明键值。

StateSpec类型的参数中封装了mapping功能和转换的相应配置(例如:partitioners、超时设定、初始状态数据等)。

mapWithState

返回的是MapWithStateDStream类型。

来看看State类。其中的注释有例子参考。

/**

* :: Experimental ::

* Abstract class for getting and updating the state in mapping function used in the `mapWithState`

* operation of a [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala)

* or a [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java).

*

* Scala example of using `State`:

* {{{

*    // A mapping function that maintains an integer state and returns a String

*    def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = {

*      // Check if state exists

*      if (state.exists) {

*        val existingState = state.get  // Get the existing state

*        val shouldRemove = ...         // Decide whether to remove the state

*        if (shouldRemove) {

*          state.remove()     // Remove the state

*        } else {

*          val newState = ...

*          state.update(newState)    // Set the new state

*        }

*      } else {

*        val initialState = ...

*        state.update(initialState)  // Set the initial state

*      }

*      ... // return something

*    }

*

* }}}

*

...

sealed abstractclassState[S] {

State中有exists、get、update、remove、isTimingOut等需要在子类中覆写的方法。

State中还有个内部实现类StateImpl:

/** Internal implementation of the [[State]] interface */

private[streaming] classStateImpl[S] extends State[S] {

private var state: S = null.asInstanceOf[S]

private var defined: Boolean = false

private var timingOut: Boolean = false

private var updated: Boolean = false

private var removed: Boolean = false

// ========= Public API =========

override def exists(): Boolean = {

defined

}

...

StateImpl有一些状态变量,并且覆写了State中的方法。

回去再看

PairDStreamFunctions中的其它mapWithState方法。

@Experimental

defmapWithState[StateType: ClassTag, MappedType: ClassTag](

spec: StateSpec[K, V, StateType, MappedType]

): MapWithStateDStream[K, V, StateType, MappedType] = {

newMapWithStateDStreamImpl[K, V, StateType, MappedType](

self,

spec.asInstanceOf[StateSpecImpl[K, V, StateType, MappedType]]

)

}

先看看StateSpecImpl。StateSpecImpl是StateSpec类中的case

class。

/** Internal implementation of [[org.apache.spark.streaming.StateSpec]] interface. */

private[streaming]

case classStateSpecImpl[K, V, S, T](

function: (Time, K, Option[V], State[S]) => Option[T])extends StateSpec[K, V, S, T] {

其参数是一个函数。

StateSpecImpl中的代码片段:

require(function != null)

@volatile private var partitioner: Partitioner = null

@volatile private var initialStateRDD: RDD[(K, S)] = null

@volatile private var timeoutInterval: Duration = null

...

// ================= Private Methods =================

private[streaming] defgetFunction(): (Time, K, Option[V], State[S]) => Option[T] =function

private[streaming] def getInitialStateRDD(): Option[RDD[(K, S)]] = Option(initialStateRDD)

private[streaming] def getPartitioner(): Option[Partitioner] = Option(partitioner)

private[streaming] def getTimeoutInterval(): Option[Duration] = Option(timeoutInterval)

有一些私有变量,及其变量的获取方法。特别是有一个函数的获取方法。

再看看MapWithStateDStream的子类MapWithStateDStreamImpl:

/** Internal implementation of the [[MapWithStateDStream]] */

private[streaming] classMapWithStateDStreamImpl[

KeyType: ClassTag, ValueType: ClassTag, StateType: ClassTag, MappedType: ClassTag](

dataStream: DStream[(KeyType, ValueType)],

spec: StateSpecImpl[KeyType, ValueType, StateType, MappedType])

extends MapWithStateDStream[KeyType, ValueType, StateType, MappedType](dataStream.context) {

private valinternalStream=

newInternalMapWithStateDStream[KeyType, ValueType, StateType, MappedType](dataStream, spec)

override def slideDuration: Duration = internalStream.slideDuration

override def dependencies: List[DStream[_]] = List(internalStream)

override defcompute(validTime: Time): Option[RDD[MappedType]] = {

internalStream.getOrCompute(validTime).map { _.flatMap[MappedType] { _.mappedData } }

}

其中生成了一个DStream子类InternalMapWithStateDStream的对象。

InternalMapWithStateDStream类:

private[streaming]

classInternalMapWithStateDStream[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag](

parent: DStream[(K, V)], spec: StateSpecImpl[K, V, S, E])

extends DStream[MapWithStateRDDRecord[K, S, E]](parent.context) {

persist(StorageLevel.MEMORY_ONLY)

InternalMapWithStateDStream.compute:

/** Method that generates a RDD for the given time */

override defcompute(validTime: Time): Option[RDD[MapWithStateRDDRecord[K, S, E]]] = {

// Get the previous state or create a new empty state RDD

val prevStateRDD = getOrCompute(validTime - slideDuration) match {

case Some(rdd) =>

if (rdd.partitioner != Some(partitioner)) {

// If the RDD is not partitioned the right way, let us repartition it using the

// partition index as the key. This is to ensure that state RDD is always partitioned

// before creating another state RDD using it

MapWithStateRDD.createFromRDD[K, V, S, E](

rdd.flatMap { _.stateMap.getAll() }, partitioner, validTime)

} else {

rdd

}

case None =>

MapWithStateRDD.createFromPairRDD[K, V, S, E](

spec.getInitialStateRDD().getOrElse(new EmptyRDD[(K, S)](ssc.sparkContext)),

partitioner,

validTime

)

}

// Compute the new state RDD with previous state RDD and partitioned data RDD

// Even if there is no data RDD, use an empty one to create a new state RDD

val dataRDD = parent.getOrCompute(validTime).getOrElse {

context.sparkContext.emptyRDD[(K, V)]

}

val partitionedDataRDD = dataRDD.partitionBy(partitioner)

val timeoutThresholdTime = spec.getTimeoutInterval().map { interval =>

(validTime - interval).milliseconds

}

Some(newMapWithStateRDD(

prevStateRDD, partitionedDataRDD, mappingFunction, validTime, timeoutThresholdTime))

}

生成了RDD子类MapWithStateRDD的对象。

MapWithStateRDD:

private[streaming] classMapWithStateRDD[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag](

private var prevStateRDD: RDD[MapWithStateRDDRecord[K, S, E]],

private var partitionedDataRDD: RDD[(K, V)],

mappingFunction: (Time, K, Option[V], State[S]) => Option[E],

batchTime: Time,

timeoutThresholdTime: Option[Long]

) extends RDD[MapWithStateRDDRecord[K, S, E]](

partitionedDataRDD.sparkContext,

List(

new OneToOneDependency[MapWithStateRDDRecord[K, S, E]](prevStateRDD),

new OneToOneDependency(partitionedDataRDD))

) {

每个RDD partition是被一个MapWithStateRDDRecord类型的记录所代表,

MapWithStateRDD.compute:

override defcompute(

partition: Partition, context: TaskContext): Iterator[MapWithStateRDDRecord[K, S, E]] = {

val stateRDDPartition = partition.asInstanceOf[MapWithStateRDDPartition]

val prevStateRDDIterator = prevStateRDD.iterator(

stateRDDPartition.previousSessionRDDPartition, context)

val dataIterator = partitionedDataRDD.iterator(

stateRDDPartition.partitionedDataRDDPartition, context)

val prevRecord = if (prevStateRDDIterator.hasNext) Some(prevStateRDDIterator.next()) else None

val newRecord = MapWithStateRDDRecord.updateRecordWithData(

prevRecord,

dataIterator,

mappingFunction,

batchTime,

timeoutThresholdTime,

removeTimedoutData = doFullScan // remove timedout data only when full scan is enabled

)

Iterator(newRecord)

}

MapWithStateRDDRecord有伴生对象:

private[streaming] objectMapWithStateRDDRecord{

defupdateRecordWithData[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag](

prevRecord: Option[MapWithStateRDDRecord[K, S, E]],

dataIterator: Iterator[(K, V)],

mappingFunction: (Time, K, Option[V], State[S]) => Option[E],

batchTime: Time,

timeoutThresholdTime: Option[Long],

removeTimedoutData: Boolean

): MapWithStateRDDRecord[K, S, E] = {

// Create a new state map by cloning the previous one (if it exists) or by creating an empty one

val newStateMap = prevRecord.map { _.stateMap.copy() }. getOrElse { new EmptyStateMap[K, S]() }

val mappedData = new ArrayBuffer[E]

val wrappedState = new StateImpl[S]()

// Call the mapping function on each record in the data iterator, and accordingly

// update the states touched, and collect the data returned by the mapping function

//

此处是mapWithState性能较好的核心代码之所在。

dataIterator.foreach { case (key, value) =>

wrappedState.wrap(newStateMap.get(key))

val returned = mappingFunction(batchTime, key, Some(value), wrappedState)

if (wrappedState.isRemoved) {

newStateMap.remove(key)

} else if (wrappedState.isUpdated

|| (wrappedState.exists && timeoutThresholdTime.isDefined)) {

newStateMap.put(key, wrappedState.get(), batchTime.milliseconds)

}

mappedData ++= returned

}

// Get the timed out state records, call the mapping function on each and collect the

// data returned

if (removeTimedoutData && timeoutThresholdTime.isDefined) {

newStateMap.getByTime(timeoutThresholdTime.get).foreach { case (key, state, _) =>

wrappedState.wrapTimingOutState(state)

val returned = mappingFunction(batchTime, key, None, wrappedState)

mappedData ++= returned

newStateMap.remove(key)

}

}

MapWithStateRDDRecord(newStateMap, mappedData)

}

借助了RDD的不变性,同时也借助了可变化特征,完成了高效的处理过程。

所以不可变的RDD也可用于处理变化的数据。


备注:

资料来源于:DT_大数据梦工厂(Spark发行版本定制)



作者:阳光男孩spark
链接:https://www.jianshu.com/p/26f8b7328984


点击查看更多内容
TA 点赞

若觉得本文不错,就分享一下吧!

评论

作者其他优质文章

正在加载中
  • 推荐
  • 评论
  • 收藏
  • 共同学习,写下你的评论
感谢您的支持,我会继续努力的~
扫码打赏,你说多少就多少
赞赏金额会直接到老师账户
支付方式
打开微信扫一扫,即可进行扫码打赏哦
今天注册有机会得

100积分直接送

付费专栏免费学

大额优惠券免费领

立即参与 放弃机会
意见反馈 帮助中心 APP下载
官方微信

举报

0/150
提交
取消