@@ -22,6 +22,7 @@ import scala.reflect.ClassTag
22
22
import org .apache .spark .annotation .Experimental
23
23
import org .apache .spark .api .java .JavaPairRDD
24
24
import org .apache .spark .rdd .RDD
25
+ import org .apache .spark .util .ClosureCleaner
25
26
import org .apache .spark .{HashPartitioner , Partitioner }
26
27
27
28
@@ -37,28 +38,33 @@ import org.apache.spark.{HashPartitioner, Partitioner}
37
38
*
38
39
* Example in Scala:
39
40
* {{{
40
- * val spec = StateSpec(trackingFunction).numPartitions(10)
41
+ * def trackingFunction(data: Option[ValueType], wrappedState: State[StateType]): EmittedType = {
42
+ * ...
43
+ * }
44
+ *
45
+ * val spec = StateSpec.function(trackingFunction).numPartitions(10)
41
46
*
42
47
* val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec)
43
48
* }}}
44
49
*
45
50
* Example in Java:
46
51
* {{{
47
- * StateStateSpec[StateType, EmittedDataType] spec =
48
- * StateStateSpec.create[StateType, EmittedDataType](trackingFunction).numPartition(10);
52
+ * StateStateSpec[KeyType, ValueType, StateType, EmittedDataType] spec =
53
+ * StateStateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction)
54
+ * .numPartition(10);
49
55
*
50
56
* JavaDStream[EmittedDataType] emittedRecordDStream =
51
57
* javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec);
52
58
* }}}
53
59
*/
54
60
@ Experimental
55
- sealed abstract class StateSpec [K , V , S , T ] extends Serializable {
61
+ sealed abstract class StateSpec [KeyType , ValueType , StateType , EmittedType ] extends Serializable {
56
62
57
63
/** Set the RDD containing the initial states that will be used by `trackStateByKey`*/
58
- def initialState (rdd : RDD [(K , S )]): this .type
64
+ def initialState (rdd : RDD [(KeyType , StateType )]): this .type
59
65
60
66
/** Set the RDD containing the initial states that will be used by `trackStateByKey`*/
61
- def initialState (javaPairRDD : JavaPairRDD [K , S ]): this .type
67
+ def initialState (javaPairRDD : JavaPairRDD [KeyType , StateType ]): this .type
62
68
63
69
/**
64
70
* Set the number of partitions by which the state RDDs generated by `trackStateByKey`
@@ -93,15 +99,20 @@ sealed abstract class StateSpec[K, V, S, T] extends Serializable {
93
99
*
94
100
* Example in Scala:
95
101
* {{{
96
- * val spec = StateSpec(trackingFunction).numPartitions(10)
102
+ * def trackingFunction(data: Option[ValueType], wrappedState: State[StateType]): EmittedType = {
103
+ * ...
104
+ * }
105
+ *
106
+ * val spec = StateSpec.function(trackingFunction).numPartitions(10)
97
107
*
98
108
* val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec)
99
109
* }}}
100
110
*
101
111
* Example in Java:
102
112
* {{{
103
- * StateStateSpec[StateType, EmittedDataType] spec =
104
- * StateStateSpec.create[StateType, EmittedDataType](trackingFunction).numPartition(10);
113
+ * StateStateSpec[KeyType, ValueType, StateType, EmittedDataType] spec =
114
+ * StateStateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction)
115
+ * .numPartition(10);
105
116
*
106
117
* JavaDStream[EmittedDataType] emittedRecordDStream =
107
118
* javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec);
@@ -115,16 +126,17 @@ object StateSpec {
115
126
* [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream ]] (Scala) or a
116
127
* [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream ]] (Java).
117
128
* @param trackingFunction The function applied on every data item to manage the associated state
118
- * and generate the emitted data and
129
+ * and generate the emitted data
119
130
* @tparam KeyType Class of the keys
120
131
* @tparam ValueType Class of the values
121
132
* @tparam StateType Class of the states data
122
133
* @tparam EmittedType Class of the emitted data
123
134
*/
124
- def apply [KeyType , ValueType , StateType , EmittedType ](
125
- trackingFunction : (KeyType , Option [ValueType ], State [StateType ]) => Option [EmittedType ]
135
+ def function [KeyType , ValueType , StateType , EmittedType ](
136
+ trackingFunction : (Time , KeyType , Option [ValueType ], State [StateType ]) => Option [EmittedType ]
126
137
): StateSpec [KeyType , ValueType , StateType , EmittedType ] = {
127
- new StateSpecImpl [KeyType , ValueType , StateType , EmittedType ](trackingFunction)
138
+ ClosureCleaner .clean(trackingFunction, checkSerializable = true )
139
+ new StateSpecImpl (trackingFunction)
128
140
}
129
141
130
142
/**
@@ -133,24 +145,28 @@ object StateSpec {
133
145
* [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream ]] (Scala) or a
134
146
* [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream ]] (Java).
135
147
* @param trackingFunction The function applied on every data item to manage the associated state
136
- * and generate the emitted data and
137
- * @tparam KeyType Class of the keys
148
+ * and generate the emitted data
138
149
* @tparam ValueType Class of the values
139
150
* @tparam StateType Class of the states data
140
151
* @tparam EmittedType Class of the emitted data
141
152
*/
142
- def create [KeyType , ValueType , StateType , EmittedType ](
143
- trackingFunction : (KeyType , Option [ValueType ], State [StateType ]) => Option [EmittedType ]
144
- ): StateSpec [KeyType , ValueType , StateType , EmittedType ] = {
145
- apply(trackingFunction)
153
+ def function [ValueType , StateType , EmittedType ](
154
+ trackingFunction : (Option [ValueType ], State [StateType ]) => EmittedType
155
+ ): StateSpec [Any , ValueType , StateType , EmittedType ] = {
156
+ ClosureCleaner .clean(trackingFunction, checkSerializable = true )
157
+ val wrappedFunction =
158
+ (time : Time , key : Any , value : Option [ValueType ], state : State [StateType ]) => {
159
+ Some (trackingFunction(value, state))
160
+ }
161
+ new StateSpecImpl [Any , ValueType , StateType , EmittedType ](wrappedFunction)
146
162
}
147
163
}
148
164
149
165
150
166
/** Internal implementation of [[org.apache.spark.streaming.StateSpec ]] interface. */
151
167
private [streaming]
152
168
case class StateSpecImpl [K , V , S , T ](
153
- function : (K , Option [V ], State [S ]) => Option [T ]) extends StateSpec [K , V , S , T ] {
169
+ function : (Time , K , Option [V ], State [S ]) => Option [T ]) extends StateSpec [K , V , S , T ] {
154
170
155
171
require(function != null )
156
172
@@ -186,7 +202,7 @@ case class StateSpecImpl[K, V, S, T](
186
202
187
203
// ================= Private Methods =================
188
204
189
- private [streaming] def getFunction (): (K , Option [V ], State [S ]) => Option [T ] = function
205
+ private [streaming] def getFunction (): (Time , K , Option [V ], State [S ]) => Option [T ] = function
190
206
191
207
private [streaming] def getInitialStateRDD (): Option [RDD [(K , S )]] = Option (initialStateRDD)
192
208
0 commit comments