Skip to content
Snippets Groups Projects
  1. Apr 28, 2015
    • jerryshao's avatar
      [SPARK-5946] [STREAMING] Add Python API for direct Kafka stream · 9e4e82b7
      jerryshao authored
      Currently only added `createDirectStream` API, I'm not sure if `createRDD` is also needed, since some Java object needs to be wrapped in Python. Please help to review, thanks a lot.
      
      Author: jerryshao <saisai.shao@intel.com>
      Author: Saisai Shao <saisai.shao@intel.com>
      
      Closes #4723 from jerryshao/direct-kafka-python-api and squashes the following commits:
      
      a1fe97c [jerryshao] Fix rebase issue
      eebf333 [jerryshao] Address the comments
      da40f4e [jerryshao] Fix Python 2.6 Syntax error issue
      5c0ee85 [jerryshao] Style fix
      4aeac18 [jerryshao] Fix bug in example code
      7146d86 [jerryshao] Add unit test
      bf3bdd6 [jerryshao] Add more APIs and address the comments
      f5b3801 [jerryshao] Small style fix
      8641835 [Saisai Shao] Rebase and update the code
      589c05b [Saisai Shao] Fix the style
      d6fcb6a [Saisai Shao] Address the comments
      dfda902 [Saisai Shao] Style fix
      0f7d168 [Saisai Shao] Add the doc and fix some style issues
      67e6880 [Saisai Shao] Fix test bug
      917b0db [Saisai Shao] Add Python createRDD API for Kakfa direct stream
      c3fc11d [jerryshao] Modify the docs
      2c00936 [Saisai Shao] address the comments
      3360f44 [jerryshao] Fix code style
      e0e0f0d [jerryshao] Code clean and bug fix
      338c41f [Saisai Shao] Add python API and example for direct kafka stream
      9e4e82b7
  2. Apr 21, 2015
    • Davies Liu's avatar
      [SPARK-6949] [SQL] [PySpark] Support Date/Timestamp in Column expression · ab9128fb
      Davies Liu authored
      This PR enable auto_convert in JavaGateway, then we could register a converter for a given types, for example, date and datetime.
      
      There are two bugs related to auto_convert, see [1] and [2], we workaround it in this PR.
      
      [1]  https://github.com/bartdag/py4j/issues/160
      [2] https://github.com/bartdag/py4j/issues/161
      
      cc rxin JoshRosen
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #5570 from davies/py4j_date and squashes the following commits:
      
      eb4fa53 [Davies Liu] fix tests in python 3
      d17d634 [Davies Liu] rollback changes in mllib
      2e7566d [Davies Liu] convert tuple into ArrayList
      ceb3779 [Davies Liu] Update rdd.py
      3c373f3 [Davies Liu] support date and datetime by auto_convert
      cb094ff [Davies Liu] enable auto convert
      ab9128fb
  3. Apr 16, 2015
    • Davies Liu's avatar
      [SPARK-4897] [PySpark] Python 3 support · 04e44b37
      Davies Liu authored
      This PR update PySpark to support Python 3 (tested with 3.4).
      
      Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped.
      
      TODO: ec2/spark-ec2.py is not fully tested with python3.
      
      Author: Davies Liu <davies@databricks.com>
      Author: twneale <twneale@gmail.com>
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #5173 from davies/python3 and squashes the following commits:
      
      d7d6323 [Davies Liu] fix tests
      6c52a98 [Davies Liu] fix mllib test
      99e334f [Davies Liu] update timeout
      b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      cafd5ec [Davies Liu] adddress comments from @mengxr
      bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      179fc8d [Davies Liu] tuning flaky tests
      8c8b957 [Davies Liu] fix ResourceWarning in Python 3
      5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      4006829 [Davies Liu] fix test
      2fc0066 [Davies Liu] add python3 path
      71535e9 [Davies Liu] fix xrange and divide
      5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      ed498c8 [Davies Liu] fix compatibility with python 3
      820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      ad7c374 [Davies Liu] fix mllib test and warning
      ef1fc2f [Davies Liu] fix tests
      4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      59bb492 [Davies Liu] fix tests
      1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      ca0fdd3 [Davies Liu] fix code style
      9563a15 [Davies Liu] add imap back for python 2
      0b1ec04 [Davies Liu] make python examples work with Python 3
      d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      a716d34 [Davies Liu] test with python 3.4
      f1700e8 [Davies Liu] fix test in python3
      671b1db [Davies Liu] fix test in python3
      692ff47 [Davies Liu] fix flaky test
      7b9699f [Davies Liu] invalidate import cache for Python 3.3+
      9c58497 [Davies Liu] fix kill worker
      309bfbf [Davies Liu] keep compatibility
      5707476 [Davies Liu] cleanup, fix hash of string in 3.3+
      8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
      f53e1f0 [Davies Liu] fix tests
      70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3
      a39167e [Davies Liu] support customize class in __main__
      814c77b [Davies Liu] run unittests with python 3
      7f4476e [Davies Liu] mllib tests passed
      d737924 [Davies Liu] pass ml tests
      375ea17 [Davies Liu] SQL tests pass
      6cc42a9 [Davies Liu] rename
      431a8de [Davies Liu] streaming tests pass
      78901a7 [Davies Liu] fix hash of serializer in Python 3
      24b2f2e [Davies Liu] pass all RDD tests
      35f48fe [Davies Liu] run future again
      1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py
      6e3c21d [Davies Liu] make cloudpickle work with Python3
      2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run
      1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out
      7354371 [twneale] buffer --> memoryview  I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work.
      b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?).
      f40d925 [twneale] xrange --> range
      e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206
      79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper
      2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3
      854be27 [Josh Rosen] Run `futurize` on Python code:
      7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py.
      04e44b37
  4. Mar 02, 2015
  5. Feb 26, 2015
    • Tathagata Das's avatar
      [SPARK-6027][SPARK-5546] Fixed --jar and --packages not working for KafkaUtils... · aa63f633
      Tathagata Das authored
      [SPARK-6027][SPARK-5546] Fixed --jar and --packages not working for KafkaUtils and improved error message
      
      The problem with SPARK-6027 in short is that JARs like the kafka-assembly.jar does not work in python as the added JAR is not visible in the classloader used by Py4J. Py4J uses Class.forName(), which does not uses the systemclassloader, but the JARs are only visible in the Thread's contextclassloader. So this back uses the context class loader to create the KafkaUtils dstream object. This works for both cases where the Kafka libraries are added with --jars spark-streaming-kafka-assembly.jar or with --packages spark-streaming-kafka
      
      Also improves the error message.
      
      davies
      
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #4779 from tdas/kafka-python-fix and squashes the following commits:
      
      fb16b04 [Tathagata Das] Removed import
      c1fdf35 [Tathagata Das] Fixed long line and improved documentation
      7b88be8 [Tathagata Das] Fixed --jar not working for KafkaUtils and improved error message
      aa63f633
  6. Feb 02, 2015
    • Davies Liu's avatar
      [SPARK-5154] [PySpark] [Streaming] Kafka streaming support in Python · 0561c454
      Davies Liu authored
      This PR brings the Python API for Spark Streaming Kafka data source.
      
      ```
          class KafkaUtils(__builtin__.object)
           |  Static methods defined here:
           |
           |  createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False,
      2), keyDecoder=<function utf8_decoder>, valueDecoder=<function utf8_decoder>)
           |      Create an input stream that pulls messages from a Kafka Broker.
           |
           |      :param ssc:  StreamingContext object
           |      :param zkQuorum:  Zookeeper quorum (hostname:port,hostname:port,..).
           |      :param groupId:  The group id for this consumer.
           |      :param topics:  Dict of (topic_name -> numPartitions) to consume.
           |                      Each partition is consumed in its own thread.
           |      :param storageLevel:  RDD storage level.
           |      :param keyDecoder:  A function used to decode key
           |      :param valueDecoder:  A function used to decode value
           |      :return: A DStream object
      ```
      run the example:
      
      ```
      bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test
      ```
      
      Author: Davies Liu <davies@databricks.com>
      Author: Tathagata Das <tdas@databricks.com>
      
      Closes #3715 from davies/kafka and squashes the following commits:
      
      d93bfe0 [Davies Liu] Update make-distribution.sh
      4280d04 [Davies Liu] address comments
      e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
      f257071 [Davies Liu] add tests for null in RDD
      23b039a [Davies Liu] address comments
      9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka
      a74da87 [Davies Liu] address comments
      dc1eed0 [Davies Liu] Update kafka_wordcount.py
      31e2317 [Davies Liu] Update kafka_wordcount.py
      370ba61 [Davies Liu] Update kafka.py
      97386b3 [Davies Liu] address comment
      2c567a5 [Davies Liu] update logging and comment
      33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
      adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api
      aea8953 [Tathagata Das] Kafka-assembly for Python API
      eea16a7 [Davies Liu] refactor
      f6ce899 [Davies Liu] add example and fix bugs
      98c8d17 [Davies Liu] fix python style
      5697a01 [Davies Liu] bypass decoder in scala
      048dbe6 [Davies Liu] fix python style
      75d485e [Davies Liu] add mqtt
      07923c4 [Davies Liu] support kafka in Python
      0561c454
Loading