@@ -29,7 +29,7 @@ import java.nio.charset.Charset
2929import java .util .concurrent .{Executors , TimeUnit , ArrayBlockingQueue }
3030import java .util .concurrent .atomic .AtomicInteger
3131
32- import scala .collection .mutable .{SynchronizedBuffer , ArrayBuffer }
32+ import scala .collection .mutable .{SynchronizedBuffer , ArrayBuffer , SynchronizedQueue }
3333
3434import com .google .common .io .Files
3535import org .scalatest .BeforeAndAfter
@@ -39,6 +39,7 @@ import org.apache.spark.storage.StorageLevel
3939import org .apache .spark .streaming .util .ManualClock
4040import org .apache .spark .util .Utils
4141import org .apache .spark .streaming .receiver .{ActorHelper , Receiver }
42+ import org .apache .spark .rdd .RDD
4243
4344class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
4445
@@ -234,6 +235,95 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
234235 logInfo(" --------------------------------" )
235236 assert(output.sum === numTotalRecords)
236237 }
238+
239+ test(" queue input stream - oneAtATime=true" ) {
240+ // Set up the streaming context and input streams
241+ val ssc = new StreamingContext (conf, batchDuration)
242+ val queue = new SynchronizedQueue [RDD [String ]]()
243+ val queueStream = ssc.queueStream(queue, oneAtATime = true )
244+ val outputBuffer = new ArrayBuffer [Seq [String ]] with SynchronizedBuffer [Seq [String ]]
245+ val outputStream = new TestOutputStream (queueStream, outputBuffer)
246+ def output = outputBuffer.filter(_.size > 0 )
247+ outputStream.register()
248+ ssc.start()
249+
250+ // Setup data queued into the stream
251+ val clock = ssc.scheduler.clock.asInstanceOf [ManualClock ]
252+ val input = Seq (" 1" , " 2" , " 3" , " 4" , " 5" )
253+ val expectedOutput = input.map(Seq (_))
254+ // Thread.sleep(1000)
255+ val inputIterator = input.toIterator
256+ for (i <- 0 until input.size) {
257+ // Enqueue more than 1 item per tick but they should dequeue one at a time
258+ inputIterator.take(2 ).foreach(i => queue += ssc.sparkContext.makeRDD(Seq (i)))
259+ clock.addToTime(batchDuration.milliseconds)
260+ }
261+ Thread .sleep(1000 )
262+ logInfo(" Stopping context" )
263+ ssc.stop()
264+
265+ // Verify whether data received was as expected
266+ logInfo(" --------------------------------" )
267+ logInfo(" output.size = " + outputBuffer.size)
268+ logInfo(" output" )
269+ outputBuffer.foreach(x => logInfo(" [" + x.mkString(" ," ) + " ]" ))
270+ logInfo(" expected output.size = " + expectedOutput.size)
271+ logInfo(" expected output" )
272+ expectedOutput.foreach(x => logInfo(" [" + x.mkString(" ," ) + " ]" ))
273+ logInfo(" --------------------------------" )
274+
275+ // Verify whether all the elements received are as expected
276+ assert(output.size === expectedOutput.size)
277+ for (i <- 0 until output.size) {
278+ assert(output(i) === expectedOutput(i))
279+ }
280+ }
281+
282+ test(" queue input stream - oneAtATime=false" ) {
283+ // Set up the streaming context and input streams
284+ val ssc = new StreamingContext (conf, batchDuration)
285+ val queue = new SynchronizedQueue [RDD [String ]]()
286+ val queueStream = ssc.queueStream(queue, oneAtATime = false )
287+ val outputBuffer = new ArrayBuffer [Seq [String ]] with SynchronizedBuffer [Seq [String ]]
288+ val outputStream = new TestOutputStream (queueStream, outputBuffer)
289+ def output = outputBuffer.filter(_.size > 0 )
290+ outputStream.register()
291+ ssc.start()
292+
293+ // Setup data queued into the stream
294+ val clock = ssc.scheduler.clock.asInstanceOf [ManualClock ]
295+ val input = Seq (" 1" , " 2" , " 3" , " 4" , " 5" )
296+ val expectedOutput = Seq (Seq (" 1" , " 2" , " 3" ), Seq (" 4" , " 5" ))
297+
298+ // Enqueue the first 3 items (one by one), they should be merged in the next batch
299+ val inputIterator = input.toIterator
300+ inputIterator.take(3 ).foreach(i => queue += ssc.sparkContext.makeRDD(Seq (i)))
301+ clock.addToTime(batchDuration.milliseconds)
302+ Thread .sleep(1000 )
303+
304+ // Enqueue the remaining items (again one by one), merged in the final batch
305+ inputIterator.foreach(i => queue += ssc.sparkContext.makeRDD(Seq (i)))
306+ clock.addToTime(batchDuration.milliseconds)
307+ Thread .sleep(1000 )
308+ logInfo(" Stopping context" )
309+ ssc.stop()
310+
311+ // Verify whether data received was as expected
312+ logInfo(" --------------------------------" )
313+ logInfo(" output.size = " + outputBuffer.size)
314+ logInfo(" output" )
315+ outputBuffer.foreach(x => logInfo(" [" + x.mkString(" ," ) + " ]" ))
316+ logInfo(" expected output.size = " + expectedOutput.size)
317+ logInfo(" expected output" )
318+ expectedOutput.foreach(x => logInfo(" [" + x.mkString(" ," ) + " ]" ))
319+ logInfo(" --------------------------------" )
320+
321+ // Verify whether all the elements received are as expected
322+ assert(output.size === expectedOutput.size)
323+ for (i <- 0 until output.size) {
324+ assert(output(i) === expectedOutput(i))
325+ }
326+ }
237327}
238328
239329
0 commit comments