@@ -25,7 +25,7 @@ import org.apache.spark._
25
25
import org .apache .spark .rdd .RDD
26
26
import org .apache .spark .api .python ._
27
27
import org .apache .spark .broadcast .Broadcast
28
- import org .apache .spark .streaming .{Duration , Time }
28
+ import org .apache .spark .streaming .{StreamingContext , Duration , Time }
29
29
import org .apache .spark .streaming .dstream ._
30
30
import org .apache .spark .streaming .api .java ._
31
31
@@ -64,7 +64,7 @@ class PythonDStream[T: ClassTag](
64
64
}
65
65
66
66
67
- private class PairwiseDStream (prev: DStream [Array [Byte ]], partitioner : Partitioner ) extends
67
+ private class PythonPairwiseDStream (prev: DStream [Array [Byte ]], partitioner : Partitioner ) extends
68
68
DStream [Array [Byte ]](prev.ssc){
69
69
override def dependencies = List (prev)
70
70
@@ -105,6 +105,7 @@ class PythonForeachDStream(
105
105
106
106
this .register()
107
107
}
108
+
108
109
/*
109
110
This does not work. Ignore this for now. -TD
110
111
class PythonTransformedDStream(
@@ -126,3 +127,30 @@ class PythonTransformedDStream(
126
127
}
127
128
*/
128
129
130
+ /**
131
+ * This is a input stream just for the unitest. This is equivalent to a checkpointable,
132
+ * replayable, reliable message queue like Kafka. It requires a sequence as input, and
133
+ * returns the i_th element at the i_th batch unde manual clock.
134
+ */
135
+ class PythonTestInputStream (ssc_ : StreamingContext , filename : String , numPartitions : Int )
136
+ extends InputDStream [Array [Byte ]](ssc_) {
137
+
138
+ def start () {}
139
+
140
+ def stop () {}
141
+
142
+ def compute (validTime : Time ): Option [RDD [Array [Byte ]]] = {
143
+ logInfo(" Computing RDD for time " + validTime)
144
+ val index = ((validTime - zeroTime) / slideDuration - 1 ).toInt
145
+ // val selectedInput = if (index < input.size) input(index) else Seq[T]()
146
+
147
+ // lets us test cases where RDDs are not created
148
+ // if (filename == null)
149
+ // return None
150
+
151
+ // val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
152
+ val rdd = PythonRDD .readRDDFromFile(ssc.sc, filename, numPartitions).rdd
153
+ logInfo(" Created RDD " + rdd.id + " with " + filename)
154
+ Some (rdd)
155
+ }
156
+ }
0 commit comments