17
17
18
18
package org .apache .spark .streaming .api .python
19
19
20
+ import java .io .{ObjectInputStream , ObjectOutputStream }
21
+ import java .lang .reflect .Proxy
20
22
import java .util .{ArrayList => JArrayList , List => JList }
21
23
import scala .collection .JavaConversions ._
22
24
import scala .collection .JavaConverters ._
23
- import scala .collection .mutable
24
25
25
26
import org .apache .spark .api .java ._
26
27
import org .apache .spark .api .python ._
@@ -35,14 +36,14 @@ import org.apache.spark.streaming.api.java._
35
36
* Interface for Python callback function with three arguments
36
37
*/
37
38
private [python] trait PythonRDDFunction {
38
- // callback in Python
39
39
def call (time : Long , rdds : JList [_]): JavaRDD [Array [Byte ]]
40
40
}
41
41
42
42
/**
43
43
* Wrapper for PythonRDDFunction
44
+ * TODO: support checkpoint
44
45
*/
45
- private [python] class RDDFunction (pfunc : PythonRDDFunction )
46
+ private [python] class RDDFunction (@ transient var pfunc : PythonRDDFunction )
46
47
extends function.Function2 [JList [JavaRDD [_]], Time , JavaRDD [Array [Byte ]]] with Serializable {
47
48
48
49
def apply (rdd : Option [RDD [_]], time : Time ): Option [RDD [Array [Byte ]]] = {
@@ -58,30 +59,62 @@ private[python] class RDDFunction(pfunc: PythonRDDFunction)
58
59
def call (rdds : JList [JavaRDD [_]], time : Time ): JavaRDD [Array [Byte ]] = {
59
60
pfunc.call(time.milliseconds, rdds)
60
61
}
61
- }
62
62
63
+ private def writeObject (out : ObjectOutputStream ): Unit = {
64
+ assert(PythonDStream .serializer != null , " Serializer has not been registered!" )
65
+ val bytes = PythonDStream .serializer.serialize(pfunc)
66
+ out.writeInt(bytes.length)
67
+ out.write(bytes)
68
+ }
69
+
70
+ private def readObject (in : ObjectInputStream ): Unit = {
71
+ assert(PythonDStream .serializer != null , " Serializer has not been registered!" )
72
+ val length = in.readInt()
73
+ val bytes = new Array [Byte ](length)
74
+ in.readFully(bytes)
75
+ pfunc = PythonDStream .serializer.deserialize(bytes)
76
+ }
77
+ }
63
78
64
79
/**
65
- * Base class for PythonDStream with some common methods
80
+ * Inferface for Python Serializer to serialize PythonRDDFunction
66
81
*/
67
- private [python]
68
- abstract class PythonDStream (parent : DStream [_], pfunc : PythonRDDFunction )
69
- extends DStream [Array [Byte ]] (parent.ssc) {
70
-
71
- val func = new RDDFunction (pfunc)
72
-
73
- override def dependencies = List (parent)
82
+ private [python] trait PythonRDDFunctionSerializer {
83
+ def dumps (id : String ): Array [Byte ] //
84
+ def loads (bytes : Array [Byte ]): PythonRDDFunction
85
+ }
74
86
75
- override def slideDuration : Duration = parent.slideDuration
87
+ /**
88
+ * Wrapper for PythonRDDFunctionSerializer
89
+ */
90
+ private [python] class RDDFunctionSerializer (pser : PythonRDDFunctionSerializer ) {
91
+ def serialize (func : PythonRDDFunction ): Array [Byte ] = {
92
+ // get the id of PythonRDDFunction in py4j
93
+ val h = Proxy .getInvocationHandler(func.asInstanceOf [Proxy ])
94
+ val f = h.getClass().getDeclaredField(" id" );
95
+ f.setAccessible(true );
96
+ val id = f.get(h).asInstanceOf [String ];
97
+ pser.dumps(id)
98
+ }
76
99
77
- val asJavaDStream = JavaDStream .fromDStream(this )
100
+ def deserialize (bytes : Array [Byte ]): PythonRDDFunction = {
101
+ pser.loads(bytes)
102
+ }
78
103
}
79
104
80
105
/**
81
106
* Helper functions
82
107
*/
83
108
private [python] object PythonDStream {
84
109
110
+ // A serializer in Python, used to serialize PythonRDDFunction
111
+ var serializer : RDDFunctionSerializer = _
112
+
113
+ // Register a serializer from Python, should be called during initialization
114
+ def registerSerializer (ser : PythonRDDFunctionSerializer ) = {
115
+ serializer = new RDDFunctionSerializer (ser)
116
+ }
117
+
85
118
// convert Option[RDD[_]] to JavaRDD, handle null gracefully
86
119
def wrapRDD (rdd : Option [RDD [_]]): JavaRDD [_] = {
87
120
if (rdd.isDefined) {
@@ -123,14 +156,30 @@ private[python] object PythonDStream {
123
156
}
124
157
}
125
158
159
+ /**
160
+ * Base class for PythonDStream with some common methods
161
+ */
162
+ private [python]
163
+ abstract class PythonDStream (parent : DStream [_], @ transient pfunc : PythonRDDFunction )
164
+ extends DStream [Array [Byte ]] (parent.ssc) {
165
+
166
+ val func = new RDDFunction (pfunc)
167
+
168
+ override def dependencies = List (parent)
169
+
170
+ override def slideDuration : Duration = parent.slideDuration
171
+
172
+ val asJavaDStream = JavaDStream .fromDStream(this )
173
+ }
174
+
126
175
/**
127
176
* Transformed DStream in Python.
128
177
*
129
178
* If `reuse` is true and the result of the `func` is an PythonRDD, then it will cache it
130
179
* as an template for future use, this can reduce the Python callbacks.
131
180
*/
132
181
private [python]
133
- class PythonTransformedDStream (parent : DStream [_], pfunc : PythonRDDFunction ,
182
+ class PythonTransformedDStream (parent : DStream [_], @ transient pfunc : PythonRDDFunction ,
134
183
var reuse : Boolean = false )
135
184
extends PythonDStream (parent, pfunc) {
136
185
@@ -170,7 +219,7 @@ class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,
170
219
*/
171
220
private [python]
172
221
class PythonTransformed2DStream (parent : DStream [_], parent2 : DStream [_],
173
- pfunc : PythonRDDFunction )
222
+ @ transient pfunc : PythonRDDFunction )
174
223
extends DStream [Array [Byte ]] (parent.ssc) {
175
224
176
225
val func = new RDDFunction (pfunc)
@@ -190,7 +239,7 @@ class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],
190
239
* similar to StateDStream
191
240
*/
192
241
private [python]
193
- class PythonStateDStream (parent : DStream [Array [Byte ]], reduceFunc : PythonRDDFunction )
242
+ class PythonStateDStream (parent : DStream [Array [Byte ]], @ transient reduceFunc : PythonRDDFunction )
194
243
extends PythonDStream (parent, reduceFunc) {
195
244
196
245
super .persist(StorageLevel .MEMORY_ONLY )
@@ -212,8 +261,8 @@ class PythonStateDStream(parent: DStream[Array[Byte]], reduceFunc: PythonRDDFunc
212
261
*/
213
262
private [python]
214
263
class PythonReducedWindowedDStream (parent : DStream [Array [Byte ]],
215
- preduceFunc : PythonRDDFunction ,
216
- pinvReduceFunc : PythonRDDFunction ,
264
+ @ transient preduceFunc : PythonRDDFunction ,
265
+ @ transient pinvReduceFunc : PythonRDDFunction ,
217
266
_windowDuration : Duration ,
218
267
_slideDuration : Duration
219
268
) extends PythonStateDStream (parent, preduceFunc) {
0 commit comments