@@ -30,7 +30,8 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
3030 parent : DStream [(K , V )],
3131 updateFunc : (Iterator [(K , Seq [V ], Option [S ])]) => Iterator [(K , S )],
3232 partitioner : Partitioner ,
33- preservePartitioning : Boolean
33+ preservePartitioning : Boolean ,
34+ initial : Option [RDD [(K , S )]]
3435 ) extends DStream [(K , S )](parent.ssc) {
3536
3637 super .persist(StorageLevel .MEMORY_ONLY_SER )
@@ -41,6 +42,28 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
4142
4243 override val mustCheckpoint = true
4344
45+ private [this ] def computeUsingPreviousRDD (
46+ parentRDD : RDD [(K , V )], prevStateRDD : RDD [(K , S )]) = {
47+ // Define the function for the mapPartition operation on cogrouped RDD;
48+ // first map the cogrouped tuple to tuples of required type,
49+ // and then apply the update function
50+ val updateFuncLocal = updateFunc
51+ val finalFunc = (iterator : Iterator [(K , (Iterable [V ], Iterable [S ]))]) => {
52+ val i = iterator.map(t => {
53+ val itr = t._2._2.iterator
54+ val headOption = itr.hasNext match {
55+ case true => Some (itr.next())
56+ case false => None
57+ }
58+ (t._1, t._2._1.toSeq, headOption)
59+ })
60+ updateFuncLocal(i)
61+ }
62+ val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner)
63+ val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning)
64+ Some (stateRDD)
65+ }
66+
4467 override def compute (validTime : Time ): Option [RDD [(K , S )]] = {
4568
4669 // Try to get the previous state RDD
@@ -51,25 +74,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
5174 // Try to get the parent RDD
5275 parent.getOrCompute(validTime) match {
5376 case Some (parentRDD) => { // If parent RDD exists, then compute as usual
54-
55- // Define the function for the mapPartition operation on cogrouped RDD;
56- // first map the cogrouped tuple to tuples of required type,
57- // and then apply the update function
58- val updateFuncLocal = updateFunc
59- val finalFunc = (iterator : Iterator [(K , (Iterable [V ], Iterable [S ]))]) => {
60- val i = iterator.map(t => {
61- val itr = t._2._2.iterator
62- val headOption = itr.hasNext match {
63- case true => Some (itr.next())
64- case false => None
65- }
66- (t._1, t._2._1.toSeq, headOption)
67- })
68- updateFuncLocal(i)
69- }
70- val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner)
71- val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning)
72- Some (stateRDD)
77+ computeUsingPreviousRDD (parentRDD, prevStateRDD)
7378 }
7479 case None => { // If parent RDD does not exist
7580
@@ -90,19 +95,25 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
9095 // Try to get the parent RDD
9196 parent.getOrCompute(validTime) match {
9297 case Some (parentRDD) => { // If parent RDD exists, then compute as usual
98+ initial match {
99+ case None => {
100+ // Define the function for the mapPartition operation on grouped RDD;
101+ // first map the grouped tuple to tuples of required type,
102+ // and then apply the update function
103+ val updateFuncLocal = updateFunc
104+ val finalFunc = (iterator : Iterator [(K , Iterable [V ])]) => {
105+ updateFuncLocal (iterator.map (tuple => (tuple._1, tuple._2.toSeq, None )))
106+ }
93107
94- // Define the function for the mapPartition operation on grouped RDD;
95- // first map the grouped tuple to tuples of required type,
96- // and then apply the update function
97- val updateFuncLocal = updateFunc
98- val finalFunc = (iterator : Iterator [(K , Iterable [V ])]) => {
99- updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2.toSeq, None )))
108+ val groupedRDD = parentRDD.groupByKey (partitioner)
109+ val sessionRDD = groupedRDD.mapPartitions (finalFunc, preservePartitioning)
110+ // logDebug("Generating state RDD for time " + validTime + " (first)")
111+ Some (sessionRDD)
112+ }
113+ case Some (initialRDD) => {
114+ computeUsingPreviousRDD(parentRDD, initialRDD)
115+ }
100116 }
101-
102- val groupedRDD = parentRDD.groupByKey(partitioner)
103- val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning)
104- // logDebug("Generating state RDD for time " + validTime + " (first)")
105- Some (sessionRDD)
106117 }
107118 case None => { // If parent RDD does not exist, then nothing to do!
108119 // logDebug("Not generating state RDD (no previous state, no parent)")
0 commit comments