aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/streaming
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-13843][STREAMING] Remove streaming-flume, streaming-mqtt, ↵Shixiong Zhu2016-03-143-473/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | streaming-zeromq, streaming-akka, streaming-twitter to Spark packages ## What changes were proposed in this pull request? Currently there are a few sub-projects, each for integrating with different external sources for Streaming. Now that we have better ability to include external libraries (spark packages) and with Spark 2.0 coming up, we can move the following projects out of Spark to https://github.com/spark-packages - streaming-flume - streaming-akka - streaming-mqtt - streaming-zeromq - streaming-twitter They are just some ancillary packages and considering the overhead of maintenance, running tests and PR failures, it's better to maintain them out of Spark. In addition, these projects can have their different release cycles and we can release them faster. I have already copied these projects to https://github.com/spark-packages ## How was this patch tested? Jenkins tests Author: Shixiong Zhu <shixiong@databricks.com> Closes #11672 from zsxwing/remove-external-pkg.
* [SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading ↵Josh Rosen2016-03-145-49/+22
| | | | | | | | | | | | | | | | issue This patch upgrades Py4J from 0.9.1 to 0.9.2 in order to include a patch which modifies Py4J to use the current thread's ContextClassLoader when performing reflection / class loading. This is necessary in order to fix [SPARK-5185](https://issues.apache.org/jira/browse/SPARK-5185), a longstanding issue affecting the use of `--jars` and `--packages` in PySpark. In order to demonstrate that the fix works, I removed the workarounds which were added as part of [SPARK-6027](https://issues.apache.org/jira/browse/SPARK-6027) / #4779 and other patches. Py4J diff: https://github.com/bartdag/py4j/compare/0.9.1...0.9.2 /cc zsxwing tdas davies brkyvz Author: Josh Rosen <joshrosen@databricks.com> Closes #11687 from JoshRosen/py4j-0.9.2.
* [SPARK-13807] De-duplicate `Python*Helper` instantiation code in PySpark ↵Josh Rosen2016-03-114-84/+60
| | | | | | | | | | | | streaming This patch de-duplicates code in PySpark streaming which loads the `Python*Helper` classes. I also changed a few `raise e` statements to simply `raise` in order to preserve the full exception stacktrace when re-throwing. Here's a link to the whitespace-change-free diff: https://github.com/apache/spark/compare/master...JoshRosen:pyspark-reflection-deduplication?w=0 Author: Josh Rosen <joshrosen@databricks.com> Closes #11641 from JoshRosen/pyspark-reflection-deduplication.
* [SPARK-13595][BUILD] Move docker, extras modules into externalSean Owen2016-03-091-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Move `docker` dirs out of top level into `external/`; move `extras/*` into `external/` ## How was this patch tested? This is tested with Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #11523 from srowen/SPARK-13595.
* [SPARK-13339][DOCS] Clarify commutative / associative operator requirements ↵Sean Owen2016-02-191-2/+2
| | | | | | | | | | | | for reduce, fold Clarify that reduce functions need to be commutative, and fold functions do not See https://github.com/apache/spark/pull/11091 Author: Sean Owen <sowen@cloudera.com> Closes #11217 from srowen/SPARK-13339.
* [SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnvShixiong Zhu2016-02-021-2/+2
| | | | | | | | | | | | `rpcEnv.awaitTermination()` was not added in #10854 because some Streaming Python tests hung forever. This patch fixed the hung issue and added rpcEnv.awaitTermination() back to SparkEnv. Previously, Streaming Kafka Python tests shutdowns the zookeeper server before stopping StreamingContext. Then when stopping StreamingContext, KafkaReceiver may be hung due to https://issues.apache.org/jira/browse/KAFKA-601, hence, some thread of RpcEnv's Dispatcher cannot exit and rpcEnv.awaitTermination is hung.The patch just changed the shutdown order to fix it. Author: Shixiong Zhu <shixiong@databricks.com> Closes #11031 from zsxwing/awaitTermination.
* [SPARK-11295][PYSPARK] Add packages to JUnit output for Python testsGábor Lipták2016-01-201-0/+1
| | | | | | | | | This is #9263 from gliptak (improving grouping/display of test case results) with a small fix of bisecting k-means unit test. Author: Gábor Lipták <gliptak@gmail.com> Author: Xiangrui Meng <meng@databricks.com> Closes #10850 from mengxr/SPARK-11295.
* Revert "[SPARK-11295] Add packages to JUnit output for Python tests"Xiangrui Meng2016-01-191-1/+0
| | | | This reverts commit c6f971b4aeca7265ab374fa46c5c452461d9b6a7.
* [SPARK-11295] Add packages to JUnit output for Python testsGábor Lipták2016-01-191-0/+1
| | | | | | | | | | SPARK-11295 Add packages to JUnit output for Python tests This improves grouping/display of test case results. Author: Gábor Lipták <gliptak@gmail.com> Closes #9263 from gliptak/SPARK-11295.
* [SPARK-12652][PYSPARK] Upgrade Py4J to 0.9.1Shixiong Zhu2016-01-122-87/+5
| | | | | | | | | | | | - [x] Upgrade Py4J to 0.9.1 - [x] SPARK-12657: Revert SPARK-12617 - [x] SPARK-12658: Revert SPARK-12511 - Still keep the change that only reading checkpoint once. This is a manual change and worth to take a look carefully. https://github.com/zsxwing/spark/commit/bfd4b5c040eb29394c3132af3c670b1a7272457c - [x] Verify no leak any more after reverting our workarounds Author: Shixiong Zhu <shixiong@databricks.com> Closes #10692 from zsxwing/py4j-0.9.1.
* [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to StreamingShixiong Zhu2016-01-061-0/+63
| | | | | | | | Move Py4jCallbackConnectionCleaner to Streaming because the callback server starts only in StreamingContext. Author: Shixiong Zhu <shixiong@databricks.com> Closes #10621 from zsxwing/SPARK-12617-2.
* [SPARK-12511] [PYSPARK] [STREAMING] Make sure ↵Shixiong Zhu2016-01-052-10/+26
| | | | | | | | | | | | PythonDStream.registerSerializer is called only once There is an issue that Py4J's PythonProxyHandler.finalize blocks forever. (https://github.com/bartdag/py4j/pull/184) Py4j will create a PythonProxyHandler in Java for "transformer_serializer" when calling "registerSerializer". If we call "registerSerializer" twice, the second PythonProxyHandler will override the first one, then the first one will be GCed and trigger "PythonProxyHandler.finalize". To avoid that, we should not call"registerSerializer" more than once, so that "PythonProxyHandler" in Java side won't be GCed. Author: Shixiong Zhu <shixiong@databricks.com> Closes #10514 from zsxwing/SPARK-12511.
* [SPARK-12353][STREAMING][PYSPARK] Fix countByValue inconsistent output in ↵jerryshao2015-12-282-5/+16
| | | | | | | | | | Python API The semantics of Python countByValue is different from Scala API, it is more like countDistinctValue, so here change to make it consistent with Scala/Java API. Author: jerryshao <sshao@hortonworks.com> Closes #10350 from jerryshao/SPARK-12353.
* [SPARK-12091] [PYSPARK] Deprecate the JAVA-specific deserialized storage levelsgatorsmile2015-12-185-7/+7
| | | | | | | | | | | | | | The current default storage level of Python persist API is MEMORY_ONLY_SER. This is different from the default level MEMORY_ONLY in the official document and RDD APIs. davies Is this inconsistency intentional? Thanks! Updates: Since the data is always serialized on the Python side, the storage levels of JAVA-specific deserialization are not removed, such as MEMORY_ONLY. Updates: Based on the reviewers' feedback. In Python, stored objects will always be serialized with the [Pickle](https://docs.python.org/2/library/pickle.html) library, so it does not matter whether you choose a serialized level. The available storage levels in Python include `MEMORY_ONLY`, `MEMORY_ONLY_2`, `MEMORY_AND_DISK`, `MEMORY_AND_DISK_2`, `DISK_ONLY`, `DISK_ONLY_2` and `OFF_HEAP`. Author: gatorsmile <gatorsmile@gmail.com> Closes #10092 from gatorsmile/persistStorageLevel.
* [SPARK-11904][PYSPARK] reduceByKeyAndWindow does not require checkpointing ↵David Tolpin2015-12-161-22/+23
| | | | | | | | | | | | | | | | when invFunc is None when invFunc is None, `reduceByKeyAndWindow(func, None, winsize, slidesize)` is equivalent to reduceByKey(func).window(winsize, slidesize).reduceByKey(winsize, slidesize) and no checkpoint is necessary. The corresponding Scala code does exactly that, but Python code always creates a windowed stream with obligatory checkpointing. The patch fixes this. I do not know how to unit-test this. Author: David Tolpin <david.tolpin@gmail.com> Closes #9888 from dtolpin/master.
* [SPARK-11713] [PYSPARK] [STREAMING] Initial RDD updateStateByKey for PySparkBryan Cutler2015-12-102-2/+31
| | | | | | | | Adding ability to define an initial state RDD for use with updateStateByKey PySpark. Added unit test and changed stateful_network_wordcount example to use initial RDD. Author: Bryan Cutler <bjcutler@us.ibm.com> Closes #10082 from BryanCutler/initial-rdd-updateStateByKey-SPARK-11713.
* [SPARK-12058][STREAMING][KINESIS][TESTS] fix Kinesis python testsBurak Yavuz2015-12-041-1/+0
| | | | | | | | | | | | Python tests require access to the `KinesisTestUtils` file. When this file exists under src/test, python can't access it, since it is not available in the assembly jar. However, if we move KinesisTestUtils to src/main, we need to add the KinesisProducerLibrary as a dependency. In order to avoid this, I moved KinesisTestUtils to src/main, and extended it with ExtendedKinesisTestUtils which is under src/test that adds support for the KPL. cc zsxwing tdas Author: Burak Yavuz <brkyvz@gmail.com> Closes #10050 from brkyvz/kinesis-py.
* [SPARK-12002][STREAMING][PYSPARK] Fix python direct stream checkpoint ↵jerryshao2015-12-012-6/+56
| | | | | | | | | | | | | recovery issue Fixed a minor race condition in #10017 Closes #10017 Author: jerryshao <sshao@hortonworks.com> Author: Shixiong Zhu <shixiong@databricks.com> Closes #10074 from zsxwing/review-pr10017.
* [SPARK-12058][HOTFIX] Disable KinesisStreamTestsShixiong Zhu2015-11-301-0/+1
| | | | | | | | | | | | KinesisStreamTests in test.py is broken because of #9403. See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46896/testReport/(root)/KinesisStreamTests/test_kinesis_stream/ Because Streaming Python didn’t work when merging https://github.com/apache/spark/pull/9403, the PR build didn’t report the Python test failure actually. This PR just disabled the test to unblock #10039 Author: Shixiong Zhu <shixiong@databricks.com> Closes #10047 from zsxwing/disable-python-kinesis-test.
* [SPARK-11935][PYSPARK] Send the Python exceptions in TransformFunction and ↵Shixiong Zhu2015-11-252-10/+101
| | | | | | | | | | | | TransformFunctionSerializer to Java The Python exception track in TransformFunction and TransformFunctionSerializer is not sent back to Java. Py4j just throws a very general exception, which is hard to debug. This PRs adds `getFailure` method to get the failure message in Java side. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9922 from zsxwing/SPARK-11935.
* [SPARK-11870][STREAMING][PYSPARK] Rethrow the exceptions in ↵Shixiong Zhu2015-11-202-0/+19
| | | | | | | | | | TransformFunction and TransformFunctionSerializer TransformFunction and TransformFunctionSerializer don't rethrow the exception, so when any exception happens, it just return None. This will cause some weird NPE and confuse people. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9847 from zsxwing/pyspark-streaming-exception.
* [SPARK-11812][PYSPARK] invFunc=None works properly with python's ↵David Tolpin2015-11-192-3/+14
| | | | | | | | | | | | | reduceByKeyAndWindow invFunc is optional and can be None. Instead of invFunc (the parameter) invReduceFunc (a local function) was checked for trueness (that is, not None, in this context). A local function is never None, thus the case of invFunc=None (a common one when inverse reduction is not defined) was treated incorrectly, resulting in loss of data. In addition, the docstring used wrong parameter names, also fixed. Author: David Tolpin <david.tolpin@gmail.com> Closes #9775 from dtolpin/master.
* [SPARK-9065][STREAMING][PYSPARK] Add MessageHandler for Kafka Python APIjerryshao2015-11-172-12/+134
| | | | | | | | | | | | Fixed the merge conflicts in #7410 Closes #7410 Author: Shixiong Zhu <shixiong@databricks.com> Author: jerryshao <saisai.shao@intel.com> Author: jerryshao <sshao@hortonworks.com> Closes #9742 from zsxwing/pr7410.
* [SPARK-11740][STREAMING] Fix the race condition of two checkpoints in a batchShixiong Zhu2015-11-171-5/+4
| | | | | | | | We will do checkpoint when generating a batch and completing a batch. When the processing time of a batch is greater than the batch interval, checkpointing for completing an old batch may run after checkpointing for generating a new batch. If this happens, checkpoint of an old batch actually has the latest information, so we want to recovery from it. This PR will use the latest checkpoint time as the file name, so that we can always recovery from the latest checkpoint file. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9707 from zsxwing/fix-checkpoint.
* [SPARK-6328][PYTHON] Python API for StreamingListenerDaniel Jalova2015-11-164-2/+210
| | | | | | Author: Daniel Jalova <djalova@us.ibm.com> Closes #9186 from djalova/SPARK-6328.
* [SPARK-11706][STREAMING] Fix the bug that Streaming Python tests cannot ↵Shixiong Zhu2015-11-131-10/+20
| | | | | | | | | | report failures This PR just checks the test results and returns 1 if the test fails, so that `run-tests.py` can mark it fail. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9669 from zsxwing/streaming-python-tests.
* [SPARK-11378][STREAMING] make StreamingContext.awaitTerminationOrTimeout ↵Nick Evans2015-11-052-1/+8
| | | | | | | | | | | | return properly This adds a failing test checking that `awaitTerminationOrTimeout` returns the expected value, and then fixes that failing test with the addition of a `return`. tdas zsxwing Author: Nick Evans <me@nicolasevans.org> Closes #9336 from manygrams/fix_await_termination_or_timeout.
* [SPARK-11270][STREAMING] Add improved equality testing for TopicAndPartition ↵Nick Evans2015-10-272-0/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | from the Kafka Streaming API jerryshao tdas I know this is kind of minor, and I know you all are busy, but this brings this class in line with the `OffsetRange` class, and makes tests a little more concise. Instead of doing something like: ``` assert topic_and_partition_instance._topic == "foo" assert topic_and_partition_instance._partition == 0 ``` You can do something like: ``` assert topic_and_partition_instance == TopicAndPartition("foo", 0) ``` Before: ``` >>> from pyspark.streaming.kafka import TopicAndPartition >>> TopicAndPartition("foo", 0) == TopicAndPartition("foo", 0) False ``` After: ``` >>> from pyspark.streaming.kafka import TopicAndPartition >>> TopicAndPartition("foo", 0) == TopicAndPartition("foo", 0) True ``` I couldn't find any tests - am I missing something? Author: Nick Evans <me@nicolasevans.org> Closes #9236 from manygrams/topic_and_partition_equality.
* [SPARK-7021] Add JUnit output for Python unit testsGábor Lipták2015-10-221-1/+10
| | | | | | | | WIP Author: Gábor Lipták <gliptak@gmail.com> Closes #8323 from gliptak/SPARK-7021.
* [SPARK-10447][SPARK-3842][PYSPARK] upgrade pyspark to py4j0.9Holden Karau2015-10-206-56/+24
| | | | | | | | | Upgrade to Py4j0.9 Author: Holden Karau <holden@pigscanfly.ca> Author: Holden Karau <holden@us.ibm.com> Closes #8615 from holdenk/SPARK-10447-upgrade-pyspark-to-py4j0.9.
* [DOC] [PYSPARK] [MLLIB] Added newlines to docstrings to fix parameter formattingnoelsmith2015-09-212-0/+3
| | | | | | | | | | | | | | Added newlines before `:param ...:` and `:return:` markup. Without these, parameter lists aren't formatted correctly in the API docs. I.e: ![screen shot 2015-09-21 at 21 49 26](https://cloud.githubusercontent.com/assets/11915197/10004686/de3c41d4-60aa-11e5-9c50-a46dcb51243f.png) .. looks like this once newline is added: ![screen shot 2015-09-21 at 21 50 14](https://cloud.githubusercontent.com/assets/11915197/10004706/f86bfb08-60aa-11e5-8524-ae4436713502.png) Author: noelsmith <mail@noelsmith.com> Closes #8851 from noel-smith/docstring-missing-newline-fix.
* [SPARK-10615] [PYSPARK] change assertEquals to assertEqualYanbo Liang2015-09-181-1/+1
| | | | | | | | As ```assertEquals``` is deprecated, so we need to change ```assertEquals``` to ```assertEqual``` for existing python unit tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #8814 from yanboliang/spark-10615.
* [SPARK-10168] [STREAMING] Fix the issue that maven publishes wrong artifact jarszsxwing2015-08-241-21/+26
| | | | | | | | | | | | | | This PR removed the `outputFile` configuration from pom.xml and updated `tests.py` to search jars for both sbt build and maven build. I ran ` mvn -Pkinesis-asl -DskipTests clean install` locally, and verified the jars in my local repository were correct. I also checked Python tests for maven build, and it passed all tests. Author: zsxwing <zsxwing@gmail.com> Closes #8373 from zsxwing/SPARK-10168 and squashes the following commits: e0b5818 [zsxwing] Fix the sbt build c697627 [zsxwing] Add the jar pathes to the exception message be1d8a5 [zsxwing] Fix the issue that maven publishes wrong artifact jars
* [SPARK-10142] [STREAMING] Made python checkpoint recovery handle non-local ↵Tathagata Das2015-08-232-16/+49
| | | | | | | | | | | | | | | | | | checkpoint paths and existing SparkContexts The current code only checks checkpoint files in local filesystem, and always tries to create a new Python SparkContext (even if one already exists). The solution is to do the following: 1. Use the same code path as Java to check whether a valid checkpoint exists 2. Create a new Python SparkContext only if there no active one. There is not test for the path as its hard to test with distributed filesystem paths in a local unit test. I am going to test it with a distributed file system manually to verify that this patch works. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8366 from tdas/SPARK-10142 and squashes the following commits: 3afa666 [Tathagata Das] Added tests 2dd4ae5 [Tathagata Das] Added the check to not create a context if one already exists 9bf151b [Tathagata Das] Made python checkpoint recovery use java to find the checkpoint files
* [SPARK-10122] [PYSPARK] [STREAMING] Fix getOffsetRanges bug in ↵jerryshao2015-08-212-2/+7
| | | | | | | | | | | | | | PySpark-Streaming transform function Details of the bug and explanations can be seen in [SPARK-10122](https://issues.apache.org/jira/browse/SPARK-10122). tdas , please help to review. Author: jerryshao <sshao@hortonworks.com> Closes #8347 from jerryshao/SPARK-10122 and squashes the following commits: 4039b16 [jerryshao] Fix getOffsetRanges in transform() bug
* [SPARK-9812] [STREAMING] Fix Python 3 compatibility issue in PySpark ↵zsxwing2015-08-193-3/+9
| | | | | | | | | | | | | Streaming and some docs This PR includes the following fixes: 1. Use `range` instead of `xrange` in `queue_stream.py` to support Python 3. 2. Fix the issue that `utf8_decoder` will return `bytes` rather than `str` when receiving an empty `bytes` in Python 3. 3. Fix the commands in docs so that the user can copy them directly to the command line. The previous commands was broken in the middle of a path, so when copying to the command line, the path would be split to two parts by the extra spaces, which forces the user to fix it manually. Author: zsxwing <zsxwing@gmail.com> Closes #8315 from zsxwing/SPARK-9812.
* [SPARK-9828] [PYSPARK] Mutable values should not be default argumentsMechCoder2015-08-141-3/+9
| | | | | | Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #8110 from MechCoder/spark-9828.
* [SPARK-9572] [STREAMING] [PYSPARK] Added ↵Tathagata Das2015-08-112-14/+176
| | | | | | | | | | | | | | | | | | | | | | | StreamingContext.getActiveOrCreate() in Python Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8080 from tdas/SPARK-9572 and squashes the following commits: 64a231d [Tathagata Das] Fix based on comments 741a0d0 [Tathagata Das] Fixed style f4f094c [Tathagata Das] Tweaked test 9afcdbe [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9572 e21488d [Tathagata Das] Minor update 1a371d9 [Tathagata Das] Addressed comments. 60479da [Tathagata Das] Fixed indent 9c2da9c [Tathagata Das] Fixed bugs b5bd32c [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9572 b55b348 [Tathagata Das] Removed prints 5781728 [Tathagata Das] Fix style issues b711214 [Tathagata Das] Reverted run-tests.py 643b59d [Tathagata Das] Revert unnecessary change 150e58c [Tathagata Das] Added StreamingContext.getActiveOrCreate() in Python
* [SPARK-9640] [STREAMING] [TEST] Do not run Python Kinesis tests when the ↵Tathagata Das2015-08-101-12/+44
| | | | | | | | | | | | | | | | | | | | Kinesis assembly JAR has not been generated Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #7961 from tdas/SPARK-9640 and squashes the following commits: 974ce19 [Tathagata Das] Undo changes related to SPARK-9727 004ae26 [Tathagata Das] style fixes 9bbb97d [Tathagata Das] Minor style fies e6a677e [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9640 ca90719 [Tathagata Das] Removed extra line ba9cfc7 [Tathagata Das] Improved kinesis test selection logic 88d59bd [Tathagata Das] updated test modules 871fcc8 [Tathagata Das] Fixed SparkBuild 94be631 [Tathagata Das] Fixed style b858196 [Tathagata Das] Fixed conditions and few other things based on PR comments. e292e64 [Tathagata Das] Added filters for Kinesis python tests
* [SPARK-5155] [PYSPARK] [STREAMING] Mqtt streaming support in PythonPrabeesh K2015-08-102-2/+176
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR is based on #4229, thanks prabeesh. Closes #4229 Author: Prabeesh K <prabsmails@gmail.com> Author: zsxwing <zsxwing@gmail.com> Author: prabs <prabsmails@gmail.com> Author: Prabeesh K <prabeesh.k@namshi.com> Closes #7833 from zsxwing/pr4229 and squashes the following commits: 9570bec [zsxwing] Fix the variable name and check null in finally 4a9c79e [zsxwing] Fix pom.xml indentation abf5f18 [zsxwing] Merge branch 'master' into pr4229 935615c [zsxwing] Fix the flaky MQTT tests 47278c5 [zsxwing] Include the project class files 478f844 [zsxwing] Add unpack 5f8a1d4 [zsxwing] Make the maven build generate the test jar for Python MQTT tests 734db99 [zsxwing] Merge branch 'master' into pr4229 126608a [Prabeesh K] address the comments b90b709 [Prabeesh K] Merge pull request #1 from zsxwing/pr4229 d07f454 [zsxwing] Register StreamingListerner before starting StreamingContext; Revert unncessary changes; fix the python unit test a6747cb [Prabeesh K] wait for starting the receiver before publishing data 87fc677 [Prabeesh K] address the comments: 97244ec [zsxwing] Make sbt build the assembly test jar for streaming mqtt 80474d1 [Prabeesh K] fix 1f0cfe9 [Prabeesh K] python style fix e1ee016 [Prabeesh K] scala style fix a5a8f9f [Prabeesh K] added Python test 9767d82 [Prabeesh K] implemented Python-friendly class a11968b [Prabeesh K] fixed python style 795ec27 [Prabeesh K] address comments ee387ae [Prabeesh K] Fix assembly jar location of mqtt-assembly 3f4df12 [Prabeesh K] updated version b34c3c1 [prabs] adress comments 3aa7fff [prabs] Added Python streaming mqtt word count example b7d42ff [prabs] Mqtt streaming support in Python
* [SPARK-8564] [STREAMING] Add the Python API for Kinesiszsxwing2015-07-312-1/+197
| | | | | | | | | | | | | | | | | | | | | | | | | This PR adds the Python API for Kinesis, including a Python example and a simple unit test. Author: zsxwing <zsxwing@gmail.com> Closes #6955 from zsxwing/kinesis-python and squashes the following commits: e42e471 [zsxwing] Merge branch 'master' into kinesis-python 455f7ea [zsxwing] Remove streaming_kinesis_asl_assembly module and simply add the source folder to streaming_kinesis_asl module 32e6451 [zsxwing] Merge remote-tracking branch 'origin/master' into kinesis-python 5082d28 [zsxwing] Fix the syntax error for Python 2.6 fca416b [zsxwing] Fix wrong comparison 96670ff [zsxwing] Fix the compilation error after merging master 756a128 [zsxwing] Merge branch 'master' into kinesis-python 6c37395 [zsxwing] Print stack trace for debug 7c5cfb0 [zsxwing] RUN_KINESIS_TESTS -> ENABLE_KINESIS_TESTS cc9d071 [zsxwing] Fix the python test errors 466b425 [zsxwing] Add python tests for Kinesis e33d505 [zsxwing] Merge remote-tracking branch 'origin/master' into kinesis-python 3da2601 [zsxwing] Fix the kinesis folder 687446b [zsxwing] Fix the error message and the maven output path add2beb [zsxwing] Merge branch 'master' into kinesis-python 4957c0b [zsxwing] Add the Python API for Kinesis
* [SPARK-8389] [STREAMING] [PYSPARK] Expose KafkaRDDs offsetRange in Pythonjerryshao2015-07-093-11/+183
| | | | | | | | | | | | | | | | | | | | | | This PR propose a simple way to expose OffsetRange in Python code, also the usage of offsetRanges is similar to Scala/Java way, here in Python we could get OffsetRange like: ``` dstream.foreachRDD(lambda r: KafkaUtils.offsetRanges(r)) ``` Reason I didn't follow the way what SPARK-8389 suggested is that: Python Kafka API has one more step to decode the message compared to Scala/Java, Which makes Python API return a transformed RDD/DStream, not directly wrapped so-called JavaKafkaRDD, so it is hard to backtrack to the original RDD to get the offsetRange. Author: jerryshao <saisai.shao@intel.com> Closes #7185 from jerryshao/SPARK-8389 and squashes the following commits: 4c6d320 [jerryshao] Another way to fix subclass deserialization issue e6a8011 [jerryshao] Address the comments fd13937 [jerryshao] Fix serialization bug 7debf1c [jerryshao] bug fix cff3893 [jerryshao] refactor the code according to the comments 2aabf9e [jerryshao] Style fix 848c708 [jerryshao] Add HasOffsetRanges for Python
* [SPARK-8378] [STREAMING] Add the Python API for Flumezsxwing2015-07-012-2/+324
| | | | | | | | | | | | | | | | | | | | | | | Author: zsxwing <zsxwing@gmail.com> Closes #6830 from zsxwing/flume-python and squashes the following commits: 78dfdac [zsxwing] Fix the compile error in the test code f1bf3c0 [zsxwing] Address TD's comments 0449723 [zsxwing] Add sbt goal streaming-flume-assembly/assembly e93736b [zsxwing] Fix the test case for determine_modules_to_test 9d5821e [zsxwing] Fix pyspark_core dependencies f9ee681 [zsxwing] Merge branch 'master' into flume-python 7a55837 [zsxwing] Add streaming_flume_assembly to run-tests.py b96b0de [zsxwing] Merge branch 'master' into flume-python ce85e83 [zsxwing] Fix incompatible issues for Python 3 01cbb3d [zsxwing] Add import sys 152364c [zsxwing] Fix the issue that StringIO doesn't work in Python 3 14ba0ff [zsxwing] Add flume-assembly for sbt building b8d5551 [zsxwing] Merge branch 'master' into flume-python 4762c34 [zsxwing] Fix the doc 0336579 [zsxwing] Refactor Flume unit tests and also add tests for Python API 9f33873 [zsxwing] Add the Python API for Flume
* [SPARK-8583] [SPARK-5482] [BUILD] Refactor python/run-tests to integrate ↵Josh Rosen2015-06-271-0/+16
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | with dev/run-tests module system This patch refactors the `python/run-tests` script: - It's now written in Python instead of Bash. - The descriptions of the tests to run are now stored in `dev/run-tests`'s modules. This allows the pull request builder to skip Python tests suites that were not affected by the pull request's changes. For example, we can now skip the PySpark Streaming test cases when only SQL files are changed. - `python/run-tests` now supports command-line flags to make it easier to run individual test suites (this addresses SPARK-5482): ``` Usage: run-tests [options] Options: -h, --help show this help message and exit --python-executables=PYTHON_EXECUTABLES A comma-separated list of Python executables to test against (default: python2.6,python3.4,pypy) --modules=MODULES A comma-separated list of Python modules to test (default: pyspark-core,pyspark-ml,pyspark-mllib ,pyspark-sql,pyspark-streaming) ``` - `dev/run-tests` has been split into multiple files: the module definitions and test utility functions are now stored inside of a `dev/sparktestsupport` Python module, allowing them to be re-used from the Python test runner script. Author: Josh Rosen <joshrosen@databricks.com> Closes #6967 from JoshRosen/run-tests-python-modules and squashes the following commits: f578d6d [Josh Rosen] Fix print for Python 2.x 8233d61 [Josh Rosen] Add python/run-tests.py to Python lint checks 34c98d2 [Josh Rosen] Fix universal_newlines for Python 3 8f65ed0 [Josh Rosen] Fix handling of module in python/run-tests 37aff00 [Josh Rosen] Python 3 fix 27a389f [Josh Rosen] Skip MLLib tests for PyPy c364ccf [Josh Rosen] Use which() to convert PYSPARK_PYTHON to an absolute path before shelling out to run tests 568a3fd [Josh Rosen] Fix hashbang 3b852ae [Josh Rosen] Fall back to PYSPARK_PYTHON when sys.executable is None (fixes a test) f53db55 [Josh Rosen] Remove python2 flag, since the test runner script also works fine under Python 3 9c80469 [Josh Rosen] Fix passing of PYSPARK_PYTHON d33e525 [Josh Rosen] Merge remote-tracking branch 'origin/master' into run-tests-python-modules 4f8902c [Josh Rosen] Python lint fixes. 8f3244c [Josh Rosen] Use universal_newlines to fix dev/run-tests doctest failures on Python 3. f542ac5 [Josh Rosen] Fix lint check for Python 3 fff4d09 [Josh Rosen] Add dev/sparktestsupport to pep8 checks 2efd594 [Josh Rosen] Update dev/run-tests to use new Python test runner flags b2ab027 [Josh Rosen] Add command-line options for running individual suites in python/run-tests caeb040 [Josh Rosen] Fixes to PySpark test module definitions d6a77d3 [Josh Rosen] Fix the tests of dev/run-tests def2d8a [Josh Rosen] Two minor fixes aec0b8f [Josh Rosen] Actually get the Kafka stuff to run properly 04015b9 [Josh Rosen] First attempt at getting PySpark Kafka test to work in new runner script 4c97136 [Josh Rosen] PYTHONPATH fixes dcc9c09 [Josh Rosen] Fix time division 32660fc [Josh Rosen] Initial cut at Python test runner refactoring 311c6a9 [Josh Rosen] Move shell utility functions to own module. 1bdeb87 [Josh Rosen] Move module definitions to separate file.
* [SPARK-8652] [PYSPARK] Check return value for all uses of doctest.testmod()Josh Rosen2015-06-261-1/+3
| | | | | | | | | | | | | | | | This patch addresses a critical issue in the PySpark tests: Several of our Python modules' `__main__` methods call `doctest.testmod()` in order to run doctests but forget to check and handle its return value. As a result, some PySpark test failures can go unnoticed because they will not fail the build. Fortunately, there was only one test failure which was masked by this bug: a `pyspark.profiler` doctest was failing due to changes in RDD pipelining. Author: Josh Rosen <joshrosen@databricks.com> Closes #7032 from JoshRosen/testmod-fix and squashes the following commits: 60dbdc0 [Josh Rosen] Account for int vs. long formatting change in Python 3 8b8d80a [Josh Rosen] Fix failing test. e6423f9 [Josh Rosen] Check return code for all uses of doctest.testmod().
* [SPARK-8444] [STREAMING] Adding Python streaming example for queueStreamBryan Cutler2015-06-191-1/+1
| | | | | | | | | | | | | A Python example similar to the existing one for Scala. Author: Bryan Cutler <bjcutler@us.ibm.com> Closes #6884 from BryanCutler/streaming-queueStream-example-8444 and squashes the following commits: 435ba7e [Bryan Cutler] [SPARK-8444] Fixed style checks, increased sleep time to show empty queue 257abb0 [Bryan Cutler] [SPARK-8444] Stop context gracefully, Removed unused import, Added description comment 376ef6e [Bryan Cutler] [SPARK-8444] Fixed bug causing DStream.pprint to append empty parenthesis to output instead of blank line 1ff5f8b [Bryan Cutler] [SPARK-8444] Adding Python streaming example for queue_stream
* [SPARK-2808] [STREAMING] [KAFKA] cleanup tests fromcody koeninger2015-06-071-5/+0
| | | | | | | | | | | | | | see if requiring producer acks eliminates the need for waitUntilLeaderOffset calls in tests Author: cody koeninger <cody@koeninger.org> Closes #5921 from koeninger/kafka-0.8.2-test-cleanup and squashes the following commits: 1e89dc8 [cody koeninger] Merge branch 'master' into kafka-0.8.2-test-cleanup 4662828 [cody koeninger] [Streaming][Kafka] filter mima issue for removal of method from private test class af1e083 [cody koeninger] Merge branch 'master' into kafka-0.8.2-test-cleanup 4298ac2 [cody koeninger] [Streaming][Kafka] update comment to trigger jenkins attempt 1274afb [cody koeninger] [Streaming][Kafka] see if requiring producer acks eliminates the need for waitUntilLeaderOffset calls in tests
* [SPARK-7497] [PYSPARK] [STREAMING] fix streaming flaky testsDavies Liu2015-06-011-8/+8
| | | | | | | | | | | | Increase the duration and timeout in streaming python tests. Author: Davies Liu <davies@databricks.com> Closes #6239 from davies/flaky_tests and squashes the following commits: d6aee8f [Davies Liu] fix window tests 26317f7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into flaky_tests 7947db6 [Davies Liu] fix streaming flaky tests
* [SPARK-6657] [PYSPARK] Fix doc warningsXiangrui Meng2015-05-181-1/+2
| | | | | | | | | | | | | | | | | | | | | | | Fixed the following warnings in `make clean html` under `python/docs`: ~~~ /Users/meng/src/spark/python/pyspark/mllib/evaluation.py:docstring of pyspark.mllib.evaluation.RankingMetrics.ndcgAt:3: ERROR: Unexpected indentation. /Users/meng/src/spark/python/pyspark/mllib/evaluation.py:docstring of pyspark.mllib.evaluation.RankingMetrics.ndcgAt:4: WARNING: Block quote ends without a blank line; unexpected unindent. /Users/meng/src/spark/python/pyspark/mllib/fpm.py:docstring of pyspark.mllib.fpm.FPGrowth.train:3: ERROR: Unexpected indentation. /Users/meng/src/spark/python/pyspark/mllib/fpm.py:docstring of pyspark.mllib.fpm.FPGrowth.train:4: WARNING: Block quote ends without a blank line; unexpected unindent. /Users/meng/src/spark/python/pyspark/sql/__init__.py:docstring of pyspark.sql.DataFrame.replace:16: WARNING: Field list ends without a blank line; unexpected unindent. /Users/meng/src/spark/python/pyspark/streaming/kafka.py:docstring of pyspark.streaming.kafka.KafkaUtils.createRDD:8: ERROR: Unexpected indentation. /Users/meng/src/spark/python/pyspark/streaming/kafka.py:docstring of pyspark.streaming.kafka.KafkaUtils.createRDD:9: WARNING: Block quote ends without a blank line; unexpected unindent. ~~~ davies Author: Xiangrui Meng <meng@databricks.com> Closes #6221 from mengxr/SPARK-6657 and squashes the following commits: e3f83fe [Xiangrui Meng] fix sql and streaming doc warnings 2b4371e [Xiangrui Meng] fix mllib python doc warnings
* [SPARK-2808][Streaming][Kafka] update kafka to 0.8.2cody koeninger2015-05-011-3/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | i don't think this should be merged until after 1.3.0 is final Author: cody koeninger <cody@koeninger.org> Author: Helena Edelson <helena.edelson@datastax.com> Closes #4537 from koeninger/wip-2808-kafka-0.8.2-upgrade and squashes the following commits: 803aa2c [cody koeninger] [SPARK-2808][Streaming][Kafka] code cleanup per TD e6dfaf6 [cody koeninger] [SPARK-2808][Streaming][Kafka] pointless whitespace change to trigger jenkins again 1770abc [cody koeninger] [SPARK-2808][Streaming][Kafka] make waitUntilLeaderOffset easier to call, call it from python tests as well d4267e9 [cody koeninger] [SPARK-2808][Streaming][Kafka] fix stderr redirect in python test script 30d991d [cody koeninger] [SPARK-2808][Streaming][Kafka] remove stderr prints since it breaks python 3 syntax 1d896e2 [cody koeninger] [SPARK-2808][Streaming][Kafka] add even even more logging to python test 4c4557f [cody koeninger] [SPARK-2808][Streaming][Kafka] add even more logging to python test 115aeee [cody koeninger] Merge branch 'master' into wip-2808-kafka-0.8.2-upgrade 2712649 [cody koeninger] [SPARK-2808][Streaming][Kafka] add more logging to python test, see why its timing out in jenkins 2b92d3f [cody koeninger] [SPARK-2808][Streaming][Kafka] wait for leader offsets in the java test as well 3824ce3 [cody koeninger] [SPARK-2808][Streaming][Kafka] naming / comments per tdas 61b3464 [cody koeninger] [SPARK-2808][Streaming][Kafka] delay for second send in boundary condition test af6f3ec [cody koeninger] [SPARK-2808][Streaming][Kafka] delay test until latest leader offset matches expected value 9edab4c [cody koeninger] [SPARK-2808][Streaming][Kafka] more shots in the dark on jenkins failing test c70ee43 [cody koeninger] [SPARK-2808][Streaming][Kafka] add more asserts to test, try to figure out why it fails on jenkins but not locally 1d10751 [cody koeninger] Merge branch 'master' into wip-2808-kafka-0.8.2-upgrade ed02d2c [cody koeninger] [SPARK-2808][Streaming][Kafka] move default argument for api version to overloaded method, for binary compat 407382e [cody koeninger] [SPARK-2808][Streaming][Kafka] update kafka to 0.8.2.1 77de6c2 [cody koeninger] Merge branch 'master' into wip-2808-kafka-0.8.2-upgrade 6953429 [cody koeninger] [SPARK-2808][Streaming][Kafka] update kafka to 0.8.2 2e67c66 [Helena Edelson] #SPARK-2808 Update to Kafka 0.8.2.0 GA from beta. d9dc2bc [Helena Edelson] Merge remote-tracking branch 'upstream/master' into wip-2808-kafka-0.8.2-upgrade e768164 [Helena Edelson] #2808 update kafka to version 0.8.2