@@ -179,22 +179,43 @@ private[streaming] class TrackStateRDD[K: ClassTag, V: ClassTag, S: ClassTag, E:
179179
180180private [streaming] object TrackStateRDD {
181181
182- def createFromPairRDD [K : ClassTag , V : ClassTag , S : ClassTag , T : ClassTag ](
182+ def createFromPairRDD [K : ClassTag , V : ClassTag , S : ClassTag , E : ClassTag ](
183183 pairRDD : RDD [(K , S )],
184184 partitioner : Partitioner ,
185- updateTime : Time ): TrackStateRDD [K , V , S , T ] = {
185+ updateTime : Time ): TrackStateRDD [K , V , S , E ] = {
186186
187187 val rddOfTrackStateRecords = pairRDD.partitionBy(partitioner).mapPartitions ({ iterator =>
188188 val stateMap = StateMap .create[K , S ](SparkEnv .get.conf)
189189 iterator.foreach { case (key, state) => stateMap.put(key, state, updateTime.milliseconds) }
190- Iterator (TrackStateRDDRecord (stateMap, Seq .empty[T ]))
190+ Iterator (TrackStateRDDRecord (stateMap, Seq .empty[E ]))
191191 }, preservesPartitioning = true )
192192
193193 val emptyDataRDD = pairRDD.sparkContext.emptyRDD[(K , V )].partitionBy(partitioner)
194194
195195 val noOpFunc = (time : Time , key : K , value : Option [V ], state : State [S ]) => None
196196
197- new TrackStateRDD [K , V , S , T ](rddOfTrackStateRecords, emptyDataRDD, noOpFunc, updateTime, None )
197+ new TrackStateRDD [K , V , S , E ](rddOfTrackStateRecords, emptyDataRDD, noOpFunc, updateTime, None )
198+ }
199+
200+ def createFromRDD [K : ClassTag , V : ClassTag , S : ClassTag , E : ClassTag ](
201+ rdd : RDD [(K , S , Long )],
202+ partitioner : Partitioner ,
203+ updateTime : Time ): TrackStateRDD [K , V , S , E ] = {
204+
205+ val pairRDD = rdd.map { x => (x._1, (x._2, x._3)) }
206+ val rddOfTrackStateRecords = pairRDD.partitionBy(partitioner).mapPartitions({ iterator =>
207+ val stateMap = StateMap .create[K , S ](SparkEnv .get.conf)
208+ iterator.foreach { case (key, (state, updateTime)) =>
209+ stateMap.put(key, state, updateTime)
210+ }
211+ Iterator (TrackStateRDDRecord (stateMap, Seq .empty[E ]))
212+ }, preservesPartitioning = true )
213+
214+ val emptyDataRDD = pairRDD.sparkContext.emptyRDD[(K , V )].partitionBy(partitioner)
215+
216+ val noOpFunc = (time : Time , key : K , value : Option [V ], state : State [S ]) => None
217+
218+ new TrackStateRDD [K , V , S , E ](rddOfTrackStateRecords, emptyDataRDD, noOpFunc, updateTime, None )
198219 }
199220}
200221
0 commit comments