aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/streaming/tests.py
diff options
context:
space:
mode:
authorgiwa <ugw.gi.world@gmail.com>2014-10-12 02:46:56 -0700
committerTathagata Das <tathagata.das1565@gmail.com>2014-10-12 02:46:56 -0700
commit69c67abaa9d4bb4b95792d1862bc65efc764c194 (patch)
tree756bd148c002a34f1277fc1ee0704c936e917920 /python/pyspark/streaming/tests.py
parent7a3f589ef86200f99624fea8322e5af0cad774a7 (diff)
downloadspark-69c67abaa9d4bb4b95792d1862bc65efc764c194.tar.gz
spark-69c67abaa9d4bb4b95792d1862bc65efc764c194.tar.bz2
spark-69c67abaa9d4bb4b95792d1862bc65efc764c194.zip
[SPARK-2377] Python API for Streaming
This patch brings Python API for Streaming. This patch is based on work from @giwa Author: giwa <ugw.gi.world@gmail.com> Author: Ken Takagiwa <ken@Kens-MacBook-Pro.local> Author: Davies Liu <davies.liu@gmail.com> Author: Ken Takagiwa <ken@kens-mbp.gateway.sonic.net> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Ken <ugw.gi.world@gmail.com> Author: Ken Takagiwa <ugw.gi.world@gmail.com> Author: Matthew Farrellee <matt@redhat.com> Closes #2538 from davies/streaming and squashes the following commits: 64561e4 [Davies Liu] fix tests 331ecce [Davies Liu] fix example 3e2492b [Davies Liu] change updateStateByKey() to easy API 182be73 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 02d0575 [Davies Liu] add wrapper for foreachRDD() bebeb4a [Davies Liu] address all comments 6db00da [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 8380064 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 52c535b [Davies Liu] remove fix for sum() e108ec1 [Davies Liu] address comments 37fe06f [Davies Liu] use random port for callback server d05871e [Davies Liu] remove reuse of PythonRDD be5e5ff [Davies Liu] merge branch of env, make tests stable. 8071541 [Davies Liu] Merge branch 'env' into streaming c7bbbce [Davies Liu] fix sphinx docs 6bb9d91 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 4d0ea8b [Davies Liu] clear reference of SparkEnv after stop 54bd92b [Davies Liu] improve tests c2b31cb [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 7a88f9f [Davies Liu] rollback RDD.setContext(), use textFileStream() to test checkpointing bd8a4c2 [Davies Liu] fix scala style 7797c70 [Davies Liu] refactor ff88bec [Davies Liu] rename RDDFunction to TransformFunction d328aca [Davies Liu] fix serializer in queueStream 6f0da2f [Davies Liu] recover from checkpoint fa7261b [Davies Liu] refactor a13ff34 [Davies Liu] address comments 8466916 [Davies Liu] support checkpoint 9a16bd1 [Davies Liu] change number of partitions during tests b98d63f [Davies Liu] change private[spark] to private[python] eed6e2a [Davies Liu] rollback not needed changes e00136b [Davies Liu] address comments 069a94c [Davies Liu] fix the number of partitions during window() 338580a [Davies Liu] change _first(), _take(), _collect() as private API 19797f9 [Davies Liu] clean up 6ebceca [Davies Liu] add more tests c40c52d [Davies Liu] change first(), take(n) to has the same behavior as RDD 98ac6c2 [Davies Liu] support ssc.transform() b983f0f [Davies Liu] address comments 847f9b9 [Davies Liu] add more docs, add first(), take() e059ca2 [Davies Liu] move check of window into Python fce0ef5 [Davies Liu] rafactor of foreachRDD() 7001b51 [Davies Liu] refactor of queueStream() 26ea396 [Davies Liu] refactor 74df565 [Davies Liu] fix print and docs b32774c [Davies Liu] move java_import into streaming 604323f [Davies Liu] enable streaming tests c499ba0 [Davies Liu] remove Time and Duration 3f0fb4b [Davies Liu] refactor fix tests c28f520 [Davies Liu] support updateStateByKey d357b70 [Davies Liu] support windowed dstream bd13026 [Davies Liu] fix examples eec401e [Davies Liu] refactor, combine TransformedRDD, fix reuse PythonRDD, fix union 9a57685 [Davies Liu] fix python style bd27874 [Davies Liu] fix scala style 7339be0 [Davies Liu] delete tests 7f53086 [Davies Liu] support transform(), refactor and cleanup df098fc [Davies Liu] Merge branch 'master' into giwa 550dfd9 [giwa] WIP fixing 1.1 merge 5cdb6fa [giwa] changed for SCCallSiteSync e685853 [giwa] meged with rebased 1.1 branch 2d32a74 [giwa] added some StreamingContextTestSuite 4a59e1e [giwa] WIP:added more test for StreamingContext 8ffdbf1 [giwa] added atexit to handle callback server d5f5fcb [giwa] added comment for StreamingContext.sparkContext 63c881a [giwa] added StreamingContext.sparkContext d39f102 [giwa] added StreamingContext.remember d542743 [giwa] clean up code 2fdf0de [Matthew Farrellee] Fix scalastyle errors c0a06bc [giwa] delete not implemented functions f385976 [giwa] delete inproper comments b0f2015 [giwa] added comment in dstream._test_output bebb3f3 [giwa] remove the last brank line fbed8da [giwa] revert pom.xml 8ed93af [giwa] fixed explanaiton 066ba90 [giwa] revert pom.xml fa4af88 [giwa] remove duplicated import 6ae3caa [giwa] revert pom.xml 7dc7391 [giwa] fixed typo 62dc7a3 [giwa] clean up exmples f04882c [giwa] clen up examples b171ec3 [giwa] fixed pep8 violation f198d14 [giwa] clean up code 3166d31 [giwa] clean up c00e091 [giwa] change test case not to use awaitTermination e80647e [giwa] adopted the latest compression way of python command 58e41ff [giwa] merge with master 455e5af [giwa] removed wasted print in DStream af336b7 [giwa] add comments ddd4ee1 [giwa] added TODO coments 99ce042 [giwa] added saveAsTextFiles and saveAsPickledFiles 2a06cdb [giwa] remove waste duplicated code c5ecfc1 [giwa] basic function test cases are passed 8dcda84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 795b2cd [giwa] broke something 1e126bf [giwa] WIP: solved partitioned and None is not recognized f67cf57 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test 953deb0 [giwa] edited the comment to add more precise description af610d3 [giwa] removed unnesessary changes c1d546e [giwa] fixed PEP-008 violation 99410be [giwa] delete waste file b3b0362 [giwa] added basic operation test cases 9cde7c9 [giwa] WIP added test case bd3ba53 [giwa] WIP 5c04a5f [giwa] WIP: added PythonTestInputStream 019ef38 [giwa] WIP 1934726 [giwa] update comment 376e3ac [giwa] WIP 932372a [giwa] clean up dstream.py 0b09cff [giwa] added stop in StreamingContext 92e333e [giwa] implemented reduce and count function in Dstream 1b83354 [giwa] Removed the waste line 88f7506 [Ken Takagiwa] Kill py4j callback server properly 54b5358 [Ken Takagiwa] tried to restart callback server 4f07163 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. fe02547 [Ken Takagiwa] remove waste file 2ad7bd3 [Ken Takagiwa] clean up codes 6197a11 [Ken Takagiwa] clean up code eb4bf48 [Ken Takagiwa] fix map function 98c2a00 [Ken Takagiwa] added count operation but this implementation need double check 58591d2 [Ken Takagiwa] reduceByKey is working 0df7111 [Ken Takagiwa] delete old file f485b1d [Ken Takagiwa] fied input of socketTextDStream dd6de81 [Ken Takagiwa] initial commit for socketTextStream 247fd74 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 4bcb318 [Ken Takagiwa] implementing transform function in Python 38adf95 [Ken Takagiwa] added reducedByKey not working yet 66fcfff [Ken Takagiwa] modify dstream.py to fix indent error 41886c2 [Ken Takagiwa] comment PythonDStream.PairwiseDStream 0b99bec [Ken] initial commit for pySparkStreaming c214199 [giwa] added testcase for combineByKey 5625bdc [giwa] added gorupByKey testcase 10ab87b [giwa] added sparkContext as input parameter in StreamingContext 10b5b04 [giwa] removed wasted print in DStream e54f986 [giwa] add comments 16aa64f [giwa] added TODO coments 74535d4 [giwa] added saveAsTextFiles and saveAsPickledFiles f76c182 [giwa] remove waste duplicated code 18c8723 [giwa] modified streaming test case to add coment 13fb44c [giwa] basic function test cases are passed 3000b2b [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 ff14070 [giwa] broke something bcdec33 [giwa] WIP: solved partitioned and None is not recognized 270a9e1 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test bb10956 [giwa] edited the comment to add more precise description 253a863 [giwa] removed unnesessary changes 3d37822 [giwa] fixed PEP-008 violation f21cab3 [giwa] delete waste file 878bad7 [giwa] added basic operation test cases ce2acd2 [giwa] WIP added test case 9ad6855 [giwa] WIP 1df77f5 [giwa] WIP: added PythonTestInputStream 1523b66 [giwa] WIP 8a0fbbc [giwa] update comment fe648e3 [giwa] WIP 29c2bc5 [giwa] initial commit for testcase 4d40d63 [giwa] clean up dstream.py c462bb3 [giwa] added stop in StreamingContext d2c01ba [giwa] clean up examples 3c45cd2 [giwa] implemented reduce and count function in Dstream b349649 [giwa] Removed the waste line 3b498e1 [Ken Takagiwa] Kill py4j callback server properly 84a9668 [Ken Takagiwa] tried to restart callback server 9ab8952 [Tathagata Das] Added extra line. 05e991b [Tathagata Das] Added missing file b1d2a30 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. 678e854 [Ken Takagiwa] remove waste file 0a8bbbb [Ken Takagiwa] clean up codes bab31c1 [Ken Takagiwa] clean up code 72b9738 [Ken Takagiwa] fix map function d3ee86a [Ken Takagiwa] added count operation but this implementation need double check 15feea9 [Ken Takagiwa] edit python sparkstreaming example 6f98e50 [Ken Takagiwa] reduceByKey is working c455c8d [Ken Takagiwa] added reducedByKey not working yet dc6995d [Ken Takagiwa] delete old file b31446a [Ken Takagiwa] fixed typo of network_workdcount.py ccfd214 [Ken Takagiwa] added doctest for pyspark.streaming.duration 0d1b954 [Ken Takagiwa] fied input of socketTextDStream f746109 [Ken Takagiwa] initial commit for socketTextStream bb7ccf3 [Ken Takagiwa] remove unused import in python 224fc5e [Ken Takagiwa] add empty line d2099d8 [Ken Takagiwa] sorted the import following Spark coding convention 5bac7ec [Ken Takagiwa] revert streaming/pom.xml e1df940 [Ken Takagiwa] revert pom.xml 494cae5 [Ken Takagiwa] remove not implemented DStream functions in python 17a74c6 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 1a0f065 [Ken Takagiwa] implementing transform function in Python d7b4d6f [Ken Takagiwa] added reducedByKey not working yet 87438e2 [Ken Takagiwa] modify dstream.py to fix indent error b406252 [Ken Takagiwa] comment PythonDStream.PairwiseDStream 454981d [Ken] initial commit for pySparkStreaming 150b94c [giwa] added some StreamingContextTestSuite f7bc8f9 [giwa] WIP:added more test for StreamingContext ee50c5a [giwa] added atexit to handle callback server fdc9125 [giwa] added comment for StreamingContext.sparkContext f5bfb70 [giwa] added StreamingContext.sparkContext da09768 [giwa] added StreamingContext.remember d68b568 [giwa] clean up code 4afa390 [giwa] clean up code 1fd6bc7 [Ken Takagiwa] Merge pull request #2 from mattf/giwa-master d9d59fe [Matthew Farrellee] Fix scalastyle errors 67473a9 [giwa] delete not implemented functions c97377c [giwa] delete inproper comments 2ea769e [giwa] added comment in dstream._test_output 3b27bd4 [giwa] remove the last brank line acfcaeb [giwa] revert pom.xml 93f7637 [giwa] fixed explanaiton 50fd6f9 [giwa] revert pom.xml 4f82c89 [giwa] remove duplicated import 9d1de23 [giwa] revert pom.xml 7339df2 [giwa] fixed typo 9c85e48 [giwa] clean up exmples 24f95db [giwa] clen up examples 0d30109 [giwa] fixed pep8 violation b7dab85 [giwa] improve test case 583e66d [giwa] move tests for streaming inside streaming directory 1d84142 [giwa] remove unimplement test f0ea311 [giwa] clean up code 171edeb [giwa] clean up 4dedd2d [giwa] change test case not to use awaitTermination 268a6a5 [giwa] Changed awaitTermination not to call awaitTermincation in Scala. Just use time.sleep instread 09a28bf [giwa] improve testcases 58150f5 [giwa] Changed the test case to focus the test operation 199e37f [giwa] adopted the latest compression way of python command 185fdbf [giwa] merge with master f1798c4 [giwa] merge with master e70f706 [giwa] added testcase for combineByKey e162822 [giwa] added gorupByKey testcase 97742fe [giwa] added sparkContext as input parameter in StreamingContext 14d4c0e [giwa] removed wasted print in DStream 6d8190a [giwa] add comments 4aa99e4 [giwa] added TODO coments e9fab72 [giwa] added saveAsTextFiles and saveAsPickledFiles 94f2b65 [giwa] remove waste duplicated code 580fbc2 [giwa] modified streaming test case to add coment 99e4bb3 [giwa] basic function test cases are passed 7051a84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 35933e1 [giwa] broke something 9767712 [giwa] WIP: solved partitioned and None is not recognized 4f2d7e6 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test 33c0f94d [giwa] edited the comment to add more precise description 774f18d [giwa] removed unnesessary changes 3a671cc [giwa] remove export PYSPARK_PYTHON in spark submit 8efa266 [giwa] fixed PEP-008 violation fa75d71 [giwa] delete waste file 7f96294 [giwa] added basic operation test cases 3dda31a [giwa] WIP added test case 1f68b78 [giwa] WIP c05922c [giwa] WIP: added PythonTestInputStream 1fd12ae [giwa] WIP c880a33 [giwa] update comment 5d22c92 [giwa] WIP ea4b06b [giwa] initial commit for testcase 5a9b525 [giwa] clean up dstream.py 79c5809 [giwa] added stop in StreamingContext 189dcea [giwa] clean up examples b8d7d24 [giwa] implemented reduce and count function in Dstream b6468e6 [giwa] Removed the waste line b47b5fd [Ken Takagiwa] Kill py4j callback server properly 19ddcdd [Ken Takagiwa] tried to restart callback server c9fc124 [Tathagata Das] Added extra line. 4caae3f [Tathagata Das] Added missing file 4eff053 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. 5e822d4 [Ken Takagiwa] remove waste file aeaf8a5 [Ken Takagiwa] clean up codes 9fa249b [Ken Takagiwa] clean up code 05459c6 [Ken Takagiwa] fix map function a9f4ecb [Ken Takagiwa] added count operation but this implementation need double check d1ee6ca [Ken Takagiwa] edit python sparkstreaming example 0b8b7d0 [Ken Takagiwa] reduceByKey is working d25d5cf [Ken Takagiwa] added reducedByKey not working yet 7f7c5d1 [Ken Takagiwa] delete old file 967dc26 [Ken Takagiwa] fixed typo of network_workdcount.py 57fb740 [Ken Takagiwa] added doctest for pyspark.streaming.duration 4b69fb1 [Ken Takagiwa] fied input of socketTextDStream 02f618a [Ken Takagiwa] initial commit for socketTextStream 4ce4058 [Ken Takagiwa] remove unused import in python 856d98e [Ken Takagiwa] add empty line 490e338 [Ken Takagiwa] sorted the import following Spark coding convention 5594bd4 [Ken Takagiwa] revert pom.xml 2adca84 [Ken Takagiwa] remove not implemented DStream functions in python e551e13 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit 3758175 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit c5518b4 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 dcf243f [Ken Takagiwa] implementing transform function in Python 9af03f4 [Ken Takagiwa] added reducedByKey not working yet 6e0d9c7 [Ken Takagiwa] modify dstream.py to fix indent error e497b9b [Ken Takagiwa] comment PythonDStream.PairwiseDStream 5c3a683 [Ken] initial commit for pySparkStreaming 665bfdb [giwa] added testcase for combineByKey a3d2379 [giwa] added gorupByKey testcase 636090a [giwa] added sparkContext as input parameter in StreamingContext e7ebb08 [giwa] removed wasted print in DStream d8b593b [giwa] add comments ea9c873 [giwa] added TODO coments 89ae38a [giwa] added saveAsTextFiles and saveAsPickledFiles e3033fc [giwa] remove waste duplicated code a14c7e1 [giwa] modified streaming test case to add coment 536def4 [giwa] basic function test cases are passed 2112638 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 080541a [giwa] broke something 0704b86 [giwa] WIP: solved partitioned and None is not recognized 90a6484 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test a65f302 [giwa] edited the comment to add more precise description bdde697 [giwa] removed unnesessary changes e8c7bfc [giwa] remove export PYSPARK_PYTHON in spark submit 3334169 [giwa] fixed PEP-008 violation db0a303 [giwa] delete waste file 2cfd3a0 [giwa] added basic operation test cases 90ae568 [giwa] WIP added test case a120d07 [giwa] WIP f671cdb [giwa] WIP: added PythonTestInputStream 56fae45 [giwa] WIP e35e101 [giwa] Merge branch 'master' into testcase ba5112d [giwa] update comment 28aa56d [giwa] WIP fb08559 [giwa] initial commit for testcase a613b85 [giwa] clean up dstream.py c40c0ef [giwa] added stop in StreamingContext 31e4260 [giwa] clean up examples d2127d6 [giwa] implemented reduce and count function in Dstream 48f7746 [giwa] Removed the waste line 0f83eaa [Ken Takagiwa] delete py4j 0.8.1 1679808 [Ken Takagiwa] Kill py4j callback server properly f96cd4e [Ken Takagiwa] tried to restart callback server fe86198 [Ken Takagiwa] add py4j 0.8.2.1 but server is not launched 1064fe0 [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark 28c6620 [Ken Takagiwa] Implemented DStream.foreachRDD in the Python API using Py4J callback server 85b0fe1 [Ken Takagiwa] Merge pull request #1 from tdas/python-foreach 54e2e8c [Tathagata Das] Added extra line. e185338 [Tathagata Das] Added missing file a778d4b [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. cc2092b [Ken Takagiwa] remove waste file d042ac6 [Ken Takagiwa] clean up codes 84a021f [Ken Takagiwa] clean up code bd20e17 [Ken Takagiwa] fix map function d01a125 [Ken Takagiwa] added count operation but this implementation need double check 7d05109 [Ken Takagiwa] merge with remote branch ae464e0 [Ken Takagiwa] edit python sparkstreaming example 04af046 [Ken Takagiwa] reduceByKey is working 3b6d7b0 [Ken Takagiwa] implementing transform function in Python 571d52d [Ken Takagiwa] added reducedByKey not working yet 5720979 [Ken Takagiwa] delete old file e604fcb [Ken Takagiwa] fixed typo of network_workdcount.py 4b7c08b [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark ce7d426 [Ken Takagiwa] added doctest for pyspark.streaming.duration a8c9fd5 [Ken Takagiwa] fixed for socketTextStream a61fa9e [Ken Takagiwa] fied input of socketTextDStream 1e84f41 [Ken Takagiwa] initial commit for socketTextStream 6d012f7 [Ken Takagiwa] remove unused import in python 25d30d5 [Ken Takagiwa] add empty line 6e0a64a [Ken Takagiwa] sorted the import following Spark coding convention fa4a7fc [Ken Takagiwa] revert streaming/pom.xml 8f8202b [Ken Takagiwa] revert streaming pom.xml c9d79dd [Ken Takagiwa] revert pom.xml 57e3e52 [Ken Takagiwa] remove not implemented DStream functions in python 0a516f5 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit a7a0b5c [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit 72bfc66 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 69e9cd3 [Ken Takagiwa] implementing transform function in Python 94a0787 [Ken Takagiwa] added reducedByKey not working yet 88068cf [Ken Takagiwa] modify dstream.py to fix indent error 1367be5 [Ken Takagiwa] comment PythonDStream.PairwiseDStream eb2b3ba [Ken] Merge remote-tracking branch 'upstream/master' d8e51f9 [Ken] initial commit for pySparkStreaming
Diffstat (limited to 'python/pyspark/streaming/tests.py')
-rw-r--r--python/pyspark/streaming/tests.py545
1 files changed, 545 insertions, 0 deletions
diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py
new file mode 100644
index 0000000000..a8d876d0fa
--- /dev/null
+++ b/python/pyspark/streaming/tests.py
@@ -0,0 +1,545 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import os
+from itertools import chain
+import time
+import operator
+import unittest
+import tempfile
+
+from pyspark.context import SparkConf, SparkContext, RDD
+from pyspark.streaming.context import StreamingContext
+
+
+class PySparkStreamingTestCase(unittest.TestCase):
+
+ timeout = 10 # seconds
+ duration = 1
+
+ def setUp(self):
+ class_name = self.__class__.__name__
+ conf = SparkConf().set("spark.default.parallelism", 1)
+ self.sc = SparkContext(appName=class_name, conf=conf)
+ self.sc.setCheckpointDir("/tmp")
+ # TODO: decrease duration to speed up tests
+ self.ssc = StreamingContext(self.sc, self.duration)
+
+ def tearDown(self):
+ self.ssc.stop()
+
+ def wait_for(self, result, n):
+ start_time = time.time()
+ while len(result) < n and time.time() - start_time < self.timeout:
+ time.sleep(0.01)
+ if len(result) < n:
+ print "timeout after", self.timeout
+
+ def _take(self, dstream, n):
+ """
+ Return the first `n` elements in the stream (will start and stop).
+ """
+ results = []
+
+ def take(_, rdd):
+ if rdd and len(results) < n:
+ results.extend(rdd.take(n - len(results)))
+
+ dstream.foreachRDD(take)
+
+ self.ssc.start()
+ self.wait_for(results, n)
+ return results
+
+ def _collect(self, dstream, n, block=True):
+ """
+ Collect each RDDs into the returned list.
+
+ :return: list, which will have the collected items.
+ """
+ result = []
+
+ def get_output(_, rdd):
+ if rdd and len(result) < n:
+ r = rdd.collect()
+ if r:
+ result.append(r)
+
+ dstream.foreachRDD(get_output)
+
+ if not block:
+ return result
+
+ self.ssc.start()
+ self.wait_for(result, n)
+ return result
+
+ def _test_func(self, input, func, expected, sort=False, input2=None):
+ """
+ @param input: dataset for the test. This should be list of lists.
+ @param func: wrapped function. This function should return PythonDStream object.
+ @param expected: expected output for this testcase.
+ """
+ if not isinstance(input[0], RDD):
+ input = [self.sc.parallelize(d, 1) for d in input]
+ input_stream = self.ssc.queueStream(input)
+ if input2 and not isinstance(input2[0], RDD):
+ input2 = [self.sc.parallelize(d, 1) for d in input2]
+ input_stream2 = self.ssc.queueStream(input2) if input2 is not None else None
+
+ # Apply test function to stream.
+ if input2:
+ stream = func(input_stream, input_stream2)
+ else:
+ stream = func(input_stream)
+
+ result = self._collect(stream, len(expected))
+ if sort:
+ self._sort_result_based_on_key(result)
+ self._sort_result_based_on_key(expected)
+ self.assertEqual(expected, result)
+
+ def _sort_result_based_on_key(self, outputs):
+ """Sort the list based on first value."""
+ for output in outputs:
+ output.sort(key=lambda x: x[0])
+
+
+class BasicOperationTests(PySparkStreamingTestCase):
+
+ def test_map(self):
+ """Basic operation test for DStream.map."""
+ input = [range(1, 5), range(5, 9), range(9, 13)]
+
+ def func(dstream):
+ return dstream.map(str)
+ expected = map(lambda x: map(str, x), input)
+ self._test_func(input, func, expected)
+
+ def test_flatMap(self):
+ """Basic operation test for DStream.faltMap."""
+ input = [range(1, 5), range(5, 9), range(9, 13)]
+
+ def func(dstream):
+ return dstream.flatMap(lambda x: (x, x * 2))
+ expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))),
+ input)
+ self._test_func(input, func, expected)
+
+ def test_filter(self):
+ """Basic operation test for DStream.filter."""
+ input = [range(1, 5), range(5, 9), range(9, 13)]
+
+ def func(dstream):
+ return dstream.filter(lambda x: x % 2 == 0)
+ expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input)
+ self._test_func(input, func, expected)
+
+ def test_count(self):
+ """Basic operation test for DStream.count."""
+ input = [range(5), range(10), range(20)]
+
+ def func(dstream):
+ return dstream.count()
+ expected = map(lambda x: [len(x)], input)
+ self._test_func(input, func, expected)
+
+ def test_reduce(self):
+ """Basic operation test for DStream.reduce."""
+ input = [range(1, 5), range(5, 9), range(9, 13)]
+
+ def func(dstream):
+ return dstream.reduce(operator.add)
+ expected = map(lambda x: [reduce(operator.add, x)], input)
+ self._test_func(input, func, expected)
+
+ def test_reduceByKey(self):
+ """Basic operation test for DStream.reduceByKey."""
+ input = [[("a", 1), ("a", 1), ("b", 1), ("b", 1)],
+ [("", 1), ("", 1), ("", 1), ("", 1)],
+ [(1, 1), (1, 1), (2, 1), (2, 1), (3, 1)]]
+
+ def func(dstream):
+ return dstream.reduceByKey(operator.add)
+ expected = [[("a", 2), ("b", 2)], [("", 4)], [(1, 2), (2, 2), (3, 1)]]
+ self._test_func(input, func, expected, sort=True)
+
+ def test_mapValues(self):
+ """Basic operation test for DStream.mapValues."""
+ input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
+ [("", 4), (1, 1), (2, 2), (3, 3)],
+ [(1, 1), (2, 1), (3, 1), (4, 1)]]
+
+ def func(dstream):
+ return dstream.mapValues(lambda x: x + 10)
+ expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)],
+ [("", 14), (1, 11), (2, 12), (3, 13)],
+ [(1, 11), (2, 11), (3, 11), (4, 11)]]
+ self._test_func(input, func, expected, sort=True)
+
+ def test_flatMapValues(self):
+ """Basic operation test for DStream.flatMapValues."""
+ input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
+ [("", 4), (1, 1), (2, 1), (3, 1)],
+ [(1, 1), (2, 1), (3, 1), (4, 1)]]
+
+ def func(dstream):
+ return dstream.flatMapValues(lambda x: (x, x + 10))
+ expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12),
+ ("c", 1), ("c", 11), ("d", 1), ("d", 11)],
+ [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)],
+ [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]]
+ self._test_func(input, func, expected)
+
+ def test_glom(self):
+ """Basic operation test for DStream.glom."""
+ input = [range(1, 5), range(5, 9), range(9, 13)]
+ rdds = [self.sc.parallelize(r, 2) for r in input]
+
+ def func(dstream):
+ return dstream.glom()
+ expected = [[[1, 2], [3, 4]], [[5, 6], [7, 8]], [[9, 10], [11, 12]]]
+ self._test_func(rdds, func, expected)
+
+ def test_mapPartitions(self):
+ """Basic operation test for DStream.mapPartitions."""
+ input = [range(1, 5), range(5, 9), range(9, 13)]
+ rdds = [self.sc.parallelize(r, 2) for r in input]
+
+ def func(dstream):
+ def f(iterator):
+ yield sum(iterator)
+ return dstream.mapPartitions(f)
+ expected = [[3, 7], [11, 15], [19, 23]]
+ self._test_func(rdds, func, expected)
+
+ def test_countByValue(self):
+ """Basic operation test for DStream.countByValue."""
+ input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]]
+
+ def func(dstream):
+ return dstream.countByValue()
+ expected = [[4], [4], [3]]
+ self._test_func(input, func, expected)
+
+ def test_groupByKey(self):
+ """Basic operation test for DStream.groupByKey."""
+ input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
+ [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
+ [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
+
+ def func(dstream):
+ return dstream.groupByKey().mapValues(list)
+
+ expected = [[(1, [1]), (2, [1]), (3, [1]), (4, [1])],
+ [(1, [1, 1, 1]), (2, [1, 1]), (3, [1])],
+ [("a", [1, 1]), ("b", [1]), ("", [1, 1, 1])]]
+ self._test_func(input, func, expected, sort=True)
+
+ def test_combineByKey(self):
+ """Basic operation test for DStream.combineByKey."""
+ input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
+ [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
+ [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
+
+ def func(dstream):
+ def add(a, b):
+ return a + str(b)
+ return dstream.combineByKey(str, add, add)
+ expected = [[(1, "1"), (2, "1"), (3, "1"), (4, "1")],
+ [(1, "111"), (2, "11"), (3, "1")],
+ [("a", "11"), ("b", "1"), ("", "111")]]
+ self._test_func(input, func, expected, sort=True)
+
+ def test_repartition(self):
+ input = [range(1, 5), range(5, 9)]
+ rdds = [self.sc.parallelize(r, 2) for r in input]
+
+ def func(dstream):
+ return dstream.repartition(1).glom()
+ expected = [[[1, 2, 3, 4]], [[5, 6, 7, 8]]]
+ self._test_func(rdds, func, expected)
+
+ def test_union(self):
+ input1 = [range(3), range(5), range(6)]
+ input2 = [range(3, 6), range(5, 6)]
+
+ def func(d1, d2):
+ return d1.union(d2)
+
+ expected = [range(6), range(6), range(6)]
+ self._test_func(input1, func, expected, input2=input2)
+
+ def test_cogroup(self):
+ input = [[(1, 1), (2, 1), (3, 1)],
+ [(1, 1), (1, 1), (1, 1), (2, 1)],
+ [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1)]]
+ input2 = [[(1, 2)],
+ [(4, 1)],
+ [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 2)]]
+
+ def func(d1, d2):
+ return d1.cogroup(d2).mapValues(lambda vs: tuple(map(list, vs)))
+
+ expected = [[(1, ([1], [2])), (2, ([1], [])), (3, ([1], []))],
+ [(1, ([1, 1, 1], [])), (2, ([1], [])), (4, ([], [1]))],
+ [("a", ([1, 1], [1, 1])), ("b", ([1], [1])), ("", ([1, 1], [1, 2]))]]
+ self._test_func(input, func, expected, sort=True, input2=input2)
+
+ def test_join(self):
+ input = [[('a', 1), ('b', 2)]]
+ input2 = [[('b', 3), ('c', 4)]]
+
+ def func(a, b):
+ return a.join(b)
+
+ expected = [[('b', (2, 3))]]
+ self._test_func(input, func, expected, True, input2)
+
+ def test_left_outer_join(self):
+ input = [[('a', 1), ('b', 2)]]
+ input2 = [[('b', 3), ('c', 4)]]
+
+ def func(a, b):
+ return a.leftOuterJoin(b)
+
+ expected = [[('a', (1, None)), ('b', (2, 3))]]
+ self._test_func(input, func, expected, True, input2)
+
+ def test_right_outer_join(self):
+ input = [[('a', 1), ('b', 2)]]
+ input2 = [[('b', 3), ('c', 4)]]
+
+ def func(a, b):
+ return a.rightOuterJoin(b)
+
+ expected = [[('b', (2, 3)), ('c', (None, 4))]]
+ self._test_func(input, func, expected, True, input2)
+
+ def test_full_outer_join(self):
+ input = [[('a', 1), ('b', 2)]]
+ input2 = [[('b', 3), ('c', 4)]]
+
+ def func(a, b):
+ return a.fullOuterJoin(b)
+
+ expected = [[('a', (1, None)), ('b', (2, 3)), ('c', (None, 4))]]
+ self._test_func(input, func, expected, True, input2)
+
+ def test_update_state_by_key(self):
+
+ def updater(vs, s):
+ if not s:
+ s = []
+ s.extend(vs)
+ return s
+
+ input = [[('k', i)] for i in range(5)]
+
+ def func(dstream):
+ return dstream.updateStateByKey(updater)
+
+ expected = [[0], [0, 1], [0, 1, 2], [0, 1, 2, 3], [0, 1, 2, 3, 4]]
+ expected = [[('k', v)] for v in expected]
+ self._test_func(input, func, expected)
+
+
+class WindowFunctionTests(PySparkStreamingTestCase):
+
+ timeout = 20
+
+ def test_window(self):
+ input = [range(1), range(2), range(3), range(4), range(5)]
+
+ def func(dstream):
+ return dstream.window(3, 1).count()
+
+ expected = [[1], [3], [6], [9], [12], [9], [5]]
+ self._test_func(input, func, expected)
+
+ def test_count_by_window(self):
+ input = [range(1), range(2), range(3), range(4), range(5)]
+
+ def func(dstream):
+ return dstream.countByWindow(3, 1)
+
+ expected = [[1], [3], [6], [9], [12], [9], [5]]
+ self._test_func(input, func, expected)
+
+ def test_count_by_window_large(self):
+ input = [range(1), range(2), range(3), range(4), range(5), range(6)]
+
+ def func(dstream):
+ return dstream.countByWindow(5, 1)
+
+ expected = [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]]
+ self._test_func(input, func, expected)
+
+ def test_count_by_value_and_window(self):
+ input = [range(1), range(2), range(3), range(4), range(5), range(6)]
+
+ def func(dstream):
+ return dstream.countByValueAndWindow(5, 1)
+
+ expected = [[1], [2], [3], [4], [5], [6], [6], [6], [6], [6]]
+ self._test_func(input, func, expected)
+
+ def test_group_by_key_and_window(self):
+ input = [[('a', i)] for i in range(5)]
+
+ def func(dstream):
+ return dstream.groupByKeyAndWindow(3, 1).mapValues(list)
+
+ expected = [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])],
+ [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]]
+ self._test_func(input, func, expected)
+
+ def test_reduce_by_invalid_window(self):
+ input1 = [range(3), range(5), range(1), range(6)]
+ d1 = self.ssc.queueStream(input1)
+ self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 0.1, 0.1))
+ self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 1, 0.1))
+
+
+class StreamingContextTests(PySparkStreamingTestCase):
+
+ duration = 0.1
+
+ def _add_input_stream(self):
+ inputs = map(lambda x: range(1, x), range(101))
+ stream = self.ssc.queueStream(inputs)
+ self._collect(stream, 1, block=False)
+
+ def test_stop_only_streaming_context(self):
+ self._add_input_stream()
+ self.ssc.start()
+ self.ssc.stop(False)
+ self.assertEqual(len(self.sc.parallelize(range(5), 5).glom().collect()), 5)
+
+ def test_stop_multiple_times(self):
+ self._add_input_stream()
+ self.ssc.start()
+ self.ssc.stop()
+ self.ssc.stop()
+
+ def test_queue_stream(self):
+ input = [range(i + 1) for i in range(3)]
+ dstream = self.ssc.queueStream(input)
+ result = self._collect(dstream, 3)
+ self.assertEqual(input, result)
+
+ def test_text_file_stream(self):
+ d = tempfile.mkdtemp()
+ self.ssc = StreamingContext(self.sc, self.duration)
+ dstream2 = self.ssc.textFileStream(d).map(int)
+ result = self._collect(dstream2, 2, block=False)
+ self.ssc.start()
+ for name in ('a', 'b'):
+ time.sleep(1)
+ with open(os.path.join(d, name), "w") as f:
+ f.writelines(["%d\n" % i for i in range(10)])
+ self.wait_for(result, 2)
+ self.assertEqual([range(10), range(10)], result)
+
+ def test_union(self):
+ input = [range(i + 1) for i in range(3)]
+ dstream = self.ssc.queueStream(input)
+ dstream2 = self.ssc.queueStream(input)
+ dstream3 = self.ssc.union(dstream, dstream2)
+ result = self._collect(dstream3, 3)
+ expected = [i * 2 for i in input]
+ self.assertEqual(expected, result)
+
+ def test_transform(self):
+ dstream1 = self.ssc.queueStream([[1]])
+ dstream2 = self.ssc.queueStream([[2]])
+ dstream3 = self.ssc.queueStream([[3]])
+
+ def func(rdds):
+ rdd1, rdd2, rdd3 = rdds
+ return rdd2.union(rdd3).union(rdd1)
+
+ dstream = self.ssc.transform([dstream1, dstream2, dstream3], func)
+
+ self.assertEqual([2, 3, 1], self._take(dstream, 3))
+
+
+class CheckpointTests(PySparkStreamingTestCase):
+
+ def setUp(self):
+ pass
+
+ def test_get_or_create(self):
+ inputd = tempfile.mkdtemp()
+ outputd = tempfile.mkdtemp() + "/"
+
+ def updater(vs, s):
+ return sum(vs, s or 0)
+
+ def setup():
+ conf = SparkConf().set("spark.default.parallelism", 1)
+ sc = SparkContext(conf=conf)
+ ssc = StreamingContext(sc, 0.5)
+ dstream = ssc.textFileStream(inputd).map(lambda x: (x, 1))
+ wc = dstream.updateStateByKey(updater)
+ wc.map(lambda x: "%s,%d" % x).saveAsTextFiles(outputd + "test")
+ wc.checkpoint(.5)
+ return ssc
+
+ cpd = tempfile.mkdtemp("test_streaming_cps")
+ self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup)
+ ssc.start()
+
+ def check_output(n):
+ while not os.listdir(outputd):
+ time.sleep(0.1)
+ time.sleep(1) # make sure mtime is larger than the previous one
+ with open(os.path.join(inputd, str(n)), 'w') as f:
+ f.writelines(["%d\n" % i for i in range(10)])
+
+ while True:
+ p = os.path.join(outputd, max(os.listdir(outputd)))
+ if '_SUCCESS' not in os.listdir(p):
+ # not finished
+ time.sleep(0.01)
+ continue
+ ordd = ssc.sparkContext.textFile(p).map(lambda line: line.split(","))
+ d = ordd.values().map(int).collect()
+ if not d:
+ time.sleep(0.01)
+ continue
+ self.assertEqual(10, len(d))
+ s = set(d)
+ self.assertEqual(1, len(s))
+ m = s.pop()
+ if n > m:
+ continue
+ self.assertEqual(n, m)
+ break
+
+ check_output(1)
+ check_output(2)
+ ssc.stop(True, True)
+
+ time.sleep(1)
+ self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup)
+ ssc.start()
+ check_output(3)
+
+
+if __name__ == "__main__":
+ unittest.main()