@@ -30,7 +30,8 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
30
30
parent : DStream [(K , V )],
31
31
updateFunc : (Iterator [(K , Seq [V ], Option [S ])]) => Iterator [(K , S )],
32
32
partitioner : Partitioner ,
33
- preservePartitioning : Boolean
33
+ preservePartitioning : Boolean ,
34
+ initial : Option [RDD [(K , S )]]
34
35
) extends DStream [(K , S )](parent.ssc) {
35
36
36
37
super .persist(StorageLevel .MEMORY_ONLY_SER )
@@ -41,6 +42,28 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
41
42
42
43
override val mustCheckpoint = true
43
44
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
+
44
67
override def compute (validTime : Time ): Option [RDD [(K , S )]] = {
45
68
46
69
// Try to get the previous state RDD
@@ -51,25 +74,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
51
74
// Try to get the parent RDD
52
75
parent.getOrCompute(validTime) match {
53
76
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)
73
78
}
74
79
case None => { // If parent RDD does not exist
75
80
@@ -90,19 +95,25 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
90
95
// Try to get the parent RDD
91
96
parent.getOrCompute(validTime) match {
92
97
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
+ }
93
107
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
+ }
100
116
}
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)
106
117
}
107
118
case None => { // If parent RDD does not exist, then nothing to do!
108
119
// logDebug("Not generating state RDD (no previous state, no parent)")
0 commit comments