@@ -20,7 +20,7 @@ package org.apache.spark.streaming
20
20
import com .google .common .base .Optional
21
21
import org .apache .spark .annotation .Experimental
22
22
import org .apache .spark .api .java .{JavaPairRDD , JavaUtils }
23
- import org .apache .spark .api .java .function .{Function2 => JFunction2 , Function4 => JFunction4 }
23
+ import org .apache .spark .api .java .function .{Function3 => JFunction3 , Function4 => JFunction4 }
24
24
import org .apache .spark .rdd .RDD
25
25
import org .apache .spark .util .ClosureCleaner
26
26
import org .apache .spark .{HashPartitioner , Partitioner }
@@ -37,8 +37,10 @@ import org.apache.spark.{HashPartitioner, Partitioner}
37
37
*
38
38
* Example in Scala:
39
39
* {{{
40
- * def mappingFunction(data: Option[ValueType], wrappedState: State[StateType]): MappedType = {
41
- * ...
40
+ * // A mapping function that maintains an integer state and return a String
41
+ * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = {
42
+ * // Use state.exists(), state.get(), state.update() and state.remove()
43
+ * // to manage state, and return the necessary string
42
44
* }
43
45
*
44
46
* val spec = StateSpec.function(mappingFunction).numPartitions(10)
@@ -48,12 +50,19 @@ import org.apache.spark.{HashPartitioner, Partitioner}
48
50
*
49
51
* Example in Java:
50
52
* {{{
51
- * StateSpec<KeyType, ValueType, StateType, MappedType> spec =
52
- * StateSpec.<KeyType, ValueType, StateType, MappedType>function(mappingFunction)
53
- * .numPartition(10);
53
+ * // A mapping function that maintains an integer state and return a string
54
+ * Function3<String, Optional<Integer>, State<Integer>, String> mappingFunction =
55
+ * new Function3<String, Optional<Integer>, State<Integer>, String>() {
56
+ * @Override
57
+ * public Optional<String> call(Optional<Integer> value, State<Integer> state) {
58
+ * // Use state.exists(), state.get(), state.update() and state.remove()
59
+ * // to manage state, and return the necessary string
60
+ * }
61
+ * };
54
62
*
55
- * JavaMapWithStateDStream<KeyType, ValueType, StateType, MappedType> mapWithStateDStream =
56
- * javaPairDStream.<StateType, MappedType>mapWithState(spec);
63
+ * JavaMapWithStateDStream<Integer, Integer, Integer, String> mapWithStateDStream =
64
+ * keyValueDStream.<Integer, String>mapWithState(
65
+ * StateSpec.function(mappingFunction).numPartitions(10));
57
66
* }}}
58
67
*
59
68
* @tparam KeyType Class of the state key
@@ -84,8 +93,8 @@ sealed abstract class StateSpec[KeyType, ValueType, StateType, MappedType] exten
84
93
85
94
/**
86
95
* Set the duration after which the state of an idle key will be removed. A key and its state is
87
- * considered idle if it has not received any data for at least the given duration. The state
88
- * tracking function will be called one final time on the idle states that are going to be
96
+ * considered idle if it has not received any data for at least the given duration. The
97
+ * mapping function will be called one final time on the idle states that are going to be
89
98
* removed; [[org.apache.spark.streaming.State State.isTimingOut() ]] set
90
99
* to `true` in that call.
91
100
*/
@@ -104,8 +113,10 @@ sealed abstract class StateSpec[KeyType, ValueType, StateType, MappedType] exten
104
113
*
105
114
* Example in Scala:
106
115
* {{{
107
- * def mappingFunction(data: Option[ValueType], wrappedState: State[StateType]): MappedType = {
108
- * ...
116
+ * // A mapping function that maintains an integer state and return a String
117
+ * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = {
118
+ * // Use state.exists(), state.get(), state.update() and state.remove()
119
+ * // to manage state, and return the necessary string
109
120
* }
110
121
*
111
122
* val spec = StateSpec.function(mappingFunction).numPartitions(10)
@@ -115,12 +126,19 @@ sealed abstract class StateSpec[KeyType, ValueType, StateType, MappedType] exten
115
126
*
116
127
* Example in Java:
117
128
* {{{
118
- * StateSpec<KeyType, ValueType, StateType, MappedType> spec =
119
- * StateSpec.<KeyType, ValueType, StateType, MappedType>function(mappingFunction)
120
- * .numPartition(10);
129
+ * // A mapping function that maintains an integer state and return a string
130
+ * Function3<String, Optional<Integer>, State<Integer>, String> mappingFunction =
131
+ * new Function3<String, Optional<Integer>, State<Integer>, String>() {
132
+ * @Override
133
+ * public Optional<String> call(Optional<Integer> value, State<Integer> state) {
134
+ * // Use state.exists(), state.get(), state.update() and state.remove()
135
+ * // to manage state, and return the necessary string
136
+ * }
137
+ * };
121
138
*
122
- * JavaMapWithStateDStream<KeyType, ValueType, StateType, MappedType> mapWithStateDStream =
123
- * javaPairDStream.<StateType, MappedType>mapWithState(spec);
139
+ * JavaMapWithStateDStream<Integer, Integer, Integer, String> mapWithStateDStream =
140
+ * keyValueDStream.<Integer, String>mapWithState(
141
+ * StateSpec.function(mappingFunction).numPartitions(10));
124
142
* }}}
125
143
*/
126
144
@ Experimental
@@ -156,12 +174,12 @@ object StateSpec {
156
174
* @tparam MappedType Class of the mapped data
157
175
*/
158
176
def function [KeyType , ValueType , StateType , MappedType ](
159
- mappingFunction : (Option [ValueType ], State [StateType ]) => MappedType
177
+ mappingFunction : (KeyType , Option [ValueType ], State [StateType ]) => MappedType
160
178
): StateSpec [KeyType , ValueType , StateType , MappedType ] = {
161
179
ClosureCleaner .clean(mappingFunction, checkSerializable = true )
162
180
val wrappedFunction =
163
- (time : Time , key : Any , value : Option [ValueType ], state : State [StateType ]) => {
164
- Some (mappingFunction(value, state))
181
+ (time : Time , key : KeyType , value : Option [ValueType ], state : State [StateType ]) => {
182
+ Some (mappingFunction(key, value, state))
165
183
}
166
184
new StateSpecImpl (wrappedFunction)
167
185
}
@@ -181,11 +199,11 @@ object StateSpec {
181
199
def function [KeyType , ValueType , StateType , MappedType ](mappingFunction :
182
200
JFunction4 [Time , KeyType , Optional [ValueType ], State [StateType ], Optional [MappedType ]]):
183
201
StateSpec [KeyType , ValueType , StateType , MappedType ] = {
184
- val trackingFunc = (time : Time , k : KeyType , v : Option [ValueType ], s : State [StateType ]) => {
202
+ val wrappedFunc = (time : Time , k : KeyType , v : Option [ValueType ], s : State [StateType ]) => {
185
203
val t = mappingFunction.call(time, k, JavaUtils .optionToOptional(v), s)
186
204
Option (t.orNull)
187
205
}
188
- StateSpec .function(trackingFunc )
206
+ StateSpec .function(wrappedFunc )
189
207
}
190
208
191
209
/**
@@ -200,12 +218,12 @@ object StateSpec {
200
218
* @tparam MappedType Class of the mapped data
201
219
*/
202
220
def function [KeyType , ValueType , StateType , MappedType ](
203
- mappingFunction : JFunction2 [ Optional [ValueType ], State [StateType ], MappedType ]):
221
+ mappingFunction : JFunction3 [ KeyType , Optional [ValueType ], State [StateType ], MappedType ]):
204
222
StateSpec [KeyType , ValueType , StateType , MappedType ] = {
205
- val trackingFunc = (v : Option [ValueType ], s : State [StateType ]) => {
206
- mappingFunction.call(Optional .fromNullable(v.get), s)
223
+ val wrappedFunc = (k : KeyType , v : Option [ValueType ], s : State [StateType ]) => {
224
+ mappingFunction.call(k, Optional .fromNullable(v.get), s)
207
225
}
208
- StateSpec .function(trackingFunc )
226
+ StateSpec .function(wrappedFunc )
209
227
}
210
228
}
211
229
0 commit comments