Commit graph

149 commits

Author SHA1 Message Date
Imran Rashid 58419b9267 [PYSPARK] Updates to pyspark broadcast 2018-09-17 14:06:09 -05:00
Xingbo Jiang ad45299d04 [SPARK-25095][PYSPARK] Python support for BarrierTaskContext
## What changes were proposed in this pull request?

Add method `barrier()` and `getTaskInfos()` in python TaskContext, these two methods are only allowed for barrier tasks.

## How was this patch tested?

Add new tests in `tests.py`

Closes #22085 from jiangxb1987/python.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-21 15:54:30 -07:00
Tathagata Das b5ccf0d395 [SPARK-24396][SS][PYSPARK] Add Structured Streaming ForeachWriter for python
## What changes were proposed in this pull request?

This PR adds `foreach` for streaming queries in Python. Users will be able to specify their processing logic in two different ways.
- As a function that takes a row as input.
- As an object that has methods `open`, `process`, and `close` methods.

See the python docs in this PR for more details.

## How was this patch tested?
Added java and python unit tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #21477 from tdas/SPARK-24396.
2018-06-15 12:56:39 -07:00
edorigatti 3e5b4ae63a [SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker

## How does this work?

The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
 - In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
 - In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.

## How was this patch tested?

Same tests, plus tests for pandas UDFs

Author: edorigatti <emilio.dorigatti@gmail.com>

Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-11 10:15:42 +08:00
Tathagata Das 223df5d9d4 [SPARK-24397][PYSPARK] Added TaskContext.getLocalProperty(key) in Python
## What changes were proposed in this pull request?

This adds a new API `TaskContext.getLocalProperty(key)` to the Python TaskContext. It mirrors the Java TaskContext API of returning a string value if the key exists, or None if the key does not exist.

## How was this patch tested?
New test added.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #21437 from tdas/SPARK-24397.
2018-05-31 11:23:57 -07:00
e-dorigatti 0ebb0c0d4d [SPARK-23754][PYTHON] Re-raising StopIteration in client code
## What changes were proposed in this pull request?

Make sure that `StopIteration`s raised in users' code do not silently interrupt processing by spark, but are raised as exceptions to the users. The users' functions are wrapped in `safe_iter` (in `shuffle.py`), which re-raises `StopIteration`s as `RuntimeError`s

## How was this patch tested?

Unit tests, making sure that the exceptions are indeed raised. I am not sure how to check whether a `Py4JJavaError` contains my exception, so I simply looked for the exception message in the java exception's `toString`. Can you propose a better way?

## License

This is my original work, licensed in the same way as spark

Author: e-dorigatti <emilio.dorigatti@gmail.com>
Author: edorigatti <emilio.dorigatti@gmail.com>

Closes #21383 from e-dorigatti/fix_spark_23754.
2018-05-30 18:11:33 +08:00
Liang-Chi Hsieh b54bbe57b3 [SPARK-24131][PYSPARK][FOLLOWUP] Add majorMinorVersion API to PySpark for determining Spark versions
## What changes were proposed in this pull request?

More close to Scala API behavior when can't parse input by throwing exception. Add tests.

## How was this patch tested?

Added tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21211 from viirya/SPARK-24131-followup.
2018-05-08 21:22:54 +08:00
Marcelo Vanzin a634d66ce7 [SPARK-24126][PYSPARK] Use build-specific temp directory for pyspark tests.
This avoids polluting and leaving garbage behind in /tmp, and allows the
usual build tools to clean up any leftover files.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21198 from vanzin/SPARK-24126.
2018-05-07 13:00:18 +08:00
hyukjinkwon f7435bec6a [SPARK-24044][PYTHON] Explicitly print out skipped tests from unittest module
## What changes were proposed in this pull request?

This PR proposes to remove duplicated dependency checking logics and also print out skipped tests from unittests.

For example, as below:

```
Skipped tests in pyspark.sql.tests with pypy:
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
...

Skipped tests in pyspark.sql.tests with python3:
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
...
```

Currently, it's not printed out in the console. I think we should better print out skipped tests in the console.

## How was this patch tested?

Manually tested. Also, fortunately, Jenkins has good environment to test the skipped output.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21107 from HyukjinKwon/skipped-tests-print.
2018-04-26 15:11:42 -07:00
hyukjinkwon fab563b9bd [SPARK-23517][PYTHON] Make pyspark.util._exception_message produce the trace from Java side by Py4JJavaError
## What changes were proposed in this pull request?

This PR proposes for `pyspark.util._exception_message` to produce the trace from Java side by `Py4JJavaError`.

Currently, in Python 2, it uses `message` attribute which `Py4JJavaError` didn't happen to have:

```python
>>> from pyspark.util import _exception_message
>>> try:
...     sc._jvm.java.lang.String(None)
... except Exception as e:
...     pass
...
>>> e.message
''
```

Seems we should use `str` instead for now:

 aa6c53b590/py4j-python/src/py4j/protocol.py (L412)

but this doesn't address the problem with non-ascii string from Java side -
 `https://github.com/bartdag/py4j/issues/306`

So, we could directly call `__str__()`:

```python
>>> e.__str__()
u'An error occurred while calling None.java.lang.String.\n: java.lang.NullPointerException\n\tat java.lang.String.<init>(String.java:588)\n\tat sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)\n\tat sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)\n\tat sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)\n\tat java.lang.reflect.Constructor.newInstance(Constructor.java:422)\n\tat py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:247)\n\tat py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\n\tat py4j.Gateway.invoke(Gateway.java:238)\n\tat py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:80)\n\tat py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:69)\n\tat py4j.GatewayConnection.run(GatewayConnection.java:214)\n\tat java.lang.Thread.run(Thread.java:745)\n'
```

which doesn't type coerce unicodes to `str` in Python 2.

This can be actually a problem:

```python
from pyspark.sql.functions import udf
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.range(1).select(udf(lambda x: [[]])()).toPandas()
```

**Before**

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/dataframe.py", line 2009, in toPandas
    raise RuntimeError("%s\n%s" % (_exception_message(e), msg))
RuntimeError:
Note: toPandas attempted Arrow optimization because 'spark.sql.execution.arrow.enabled' is set to true. Please set it to false to disable this.
```

**After**

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/dataframe.py", line 2009, in toPandas
    raise RuntimeError("%s\n%s" % (_exception_message(e), msg))
RuntimeError: An error occurred while calling o47.collectAsArrowToPython.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 7 in stage 0.0 failed 1 times, most recent failure: Lost task 7.0 in stage 0.0 (TID 7, localhost, executor driver): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/.../spark/python/pyspark/worker.py", line 245, in main
    process()
  File "/.../spark/python/pyspark/worker.py", line 240, in process
...
Note: toPandas attempted Arrow optimization because 'spark.sql.execution.arrow.enabled' is set to true. Please set it to false to disable this.
```

## How was this patch tested?

Manually tested and unit tests were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20680 from HyukjinKwon/SPARK-23517.
2018-03-01 00:44:13 +09:00
王晓哲 602c6d82d8 [SPARK-20947][PYTHON] Fix encoding/decoding error in pipe action
## What changes were proposed in this pull request?

Pipe action convert objects into strings using a way that was affected by the default encoding setting of Python environment.

This patch fixed the problem. The detailed description is added here:

https://issues.apache.org/jira/browse/SPARK-20947

## How was this patch tested?

Run the following statement in pyspark-shell, and it will NOT raise exception if this patch is applied:

```python
sc.parallelize([u'\u6d4b\u8bd5']).pipe('cat').collect()
```

Author: 王晓哲 <wxz@linkdoc.com>

Closes #18277 from chaoslawful/fix_pipe_encoding_error.
2018-01-22 10:43:12 +09:00
hyukjinkwon 7c7266208a [SPARK-22043][PYTHON] Improves error message for show_profiles and dump_profiles
## What changes were proposed in this pull request?

This PR proposes to improve error message from:

```
>>> sc.show_profiles()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/context.py", line 1000, in show_profiles
    self.profiler_collector.show_profiles()
AttributeError: 'NoneType' object has no attribute 'show_profiles'
>>> sc.dump_profiles("/tmp/abc")
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/context.py", line 1005, in dump_profiles
    self.profiler_collector.dump_profiles(path)
AttributeError: 'NoneType' object has no attribute 'dump_profiles'
```

to

```
>>> sc.show_profiles()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/context.py", line 1003, in show_profiles
    raise RuntimeError("'spark.python.profile' configuration must be set "
RuntimeError: 'spark.python.profile' configuration must be set to 'true' to enable Python profile.
>>> sc.dump_profiles("/tmp/abc")
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/context.py", line 1012, in dump_profiles
    raise RuntimeError("'spark.python.profile' configuration must be set "
RuntimeError: 'spark.python.profile' configuration must be set to 'true' to enable Python profile.
```

## How was this patch tested?

Unit tests added in `python/pyspark/tests.py` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19260 from HyukjinKwon/profile-errors.
2017-09-18 13:20:11 +09:00
Andrew Ray 6adf67dd14 [SPARK-21985][PYSPARK] PairDeserializer is broken for double-zipped RDDs
## What changes were proposed in this pull request?
(edited)
Fixes a bug introduced in #16121

In PairDeserializer convert each batch of keys and values to lists (if they do not have `__len__` already) so that we can check that they are the same size. Normally they already are lists so this should not have a performance impact, but this is needed when repeated `zip`'s are done.

## How was this patch tested?

Additional unit test

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #19226 from aray/SPARK-21985.
2017-09-18 02:46:27 +09:00
Bryan Cutler 77cc0d67d5 [SPARK-12717][PYTHON] Adding thread-safe broadcast pickle registry
## What changes were proposed in this pull request?

When using PySpark broadcast variables in a multi-threaded environment,  `SparkContext._pickled_broadcast_vars` becomes a shared resource.  A race condition can occur when broadcast variables that are pickled from one thread get added to the shared ` _pickled_broadcast_vars` and become part of the python command from another thread.  This PR introduces a thread-safe pickled registry using thread local storage so that when python command is pickled (causing the broadcast variable to be pickled and added to the registry) each thread will have their own view of the pickle registry to retrieve and clear the broadcast variables used.

## How was this patch tested?

Added a unit test that causes this race condition using another thread.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #18695 from BryanCutler/pyspark-bcast-threadsafe-SPARK-12717.
2017-08-02 07:12:23 +09:00
chie8842 c3713fde86 [SPARK-21358][EXAMPLES] Argument of repartitionandsortwithinpartitions at pyspark
## What changes were proposed in this pull request?
At example of repartitionAndSortWithinPartitions at rdd.py, third argument should be True or False.
I proposed fix of example code.

## How was this patch tested?
* I rename test_repartitionAndSortWithinPartitions to test_repartitionAndSortWIthinPartitions_asc to specify boolean argument.
* I added test_repartitionAndSortWithinPartitions_desc to test False pattern at third argument.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: chie8842 <chie8842@gmail.com>

Closes #18586 from chie8842/SPARK-21358.
2017-07-10 18:56:54 -07:00
hyukjinkwon 7387126f83 [SPARK-19872] [PYTHON] Use the correct deserializer for RDD construction for coalesce/repartition
## What changes were proposed in this pull request?

This PR proposes to use the correct deserializer, `BatchedSerializer` for RDD construction for coalesce/repartition when the shuffle is enabled. Currently, it is passing `UTF8Deserializer` as is not `BatchedSerializer` from the copied one.

with the file, `text.txt` below:

```
a
b

d
e
f
g
h
i
j
k
l

```

- Before

```python
>>> sc.textFile('text.txt').repartition(1).collect()
```

```
UTF8Deserializer(True)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/rdd.py", line 811, in collect
    return list(_load_from_socket(port, self._jrdd_deserializer))
  File ".../spark/python/pyspark/serializers.py", line 549, in load_stream
    yield self.loads(stream)
  File ".../spark/python/pyspark/serializers.py", line 544, in loads
    return s.decode("utf-8") if self.use_unicode else s
  File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/encodings/utf_8.py", line 16, in decode
    return codecs.utf_8_decode(input, errors, True)
UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 0: invalid start byte
```

- After

```python
>>> sc.textFile('text.txt').repartition(1).collect()
```

```
[u'a', u'b', u'', u'd', u'e', u'f', u'g', u'h', u'i', u'j', u'k', u'l', u'']
```

## How was this patch tested?

Unit test in `python/pyspark/tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17282 from HyukjinKwon/SPARK-19872.
2017-03-15 10:17:18 -07:00
Bryan Cutler 44281ca81d [SPARK-19348][PYTHON] PySpark keyword_only decorator is not thread-safe
## What changes were proposed in this pull request?
The `keyword_only` decorator in PySpark is not thread-safe.  It writes kwargs to a static class variable in the decorator, which is then retrieved later in the class method as `_input_kwargs`.  If multiple threads are constructing the same class with different kwargs, it becomes a race condition to read from the static class variable before it's overwritten.  See [SPARK-19348](https://issues.apache.org/jira/browse/SPARK-19348) for reproduction code.

This change will write the kwargs to a member variable so that multiple threads can operate on separate instances without the race condition.  It does not protect against multiple threads operating on a single instance, but that is better left to the user to synchronize.

## How was this patch tested?
Added new unit tests for using the keyword_only decorator and a regression test that verifies `_input_kwargs` can be overwritten from different class instances.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #16782 from BryanCutler/pyspark-keyword_only-threadsafe-SPARK-19348.
2017-03-03 16:43:45 -08:00
Yuming Wang 9b8eca65dc [SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6
## What changes were proposed in this pull request?

Replace all the Hadoop deprecated configuration property names according to [DeprecatedProperties](https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/DeprecatedProperties.html).

except:
https://github.com/apache/spark/blob/v2.1.0/python/pyspark/sql/tests.py#L1533
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L987
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala#L45
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L614

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16990 from wangyum/HadoopDeprecatedProperties.
2017-02-28 10:13:42 +00:00
Marcelo Vanzin 92afaa93a0 [SPARK-19307][PYSPARK] Make sure user conf is propagated to SparkContext.
The code was failing to propagate the user conf in the case where the
JVM was already initialized, which happens when a user submits a
python script via spark-submit.

Tested with new unit test and by running a python script in a real cluster.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16682 from vanzin/SPARK-19307.
2017-01-25 12:08:08 -08:00
Holden Karau 047a9d92ca [SPARK-18576][PYTHON] Add basic TaskContext information to PySpark
## What changes were proposed in this pull request?

Adds basic TaskContext information to PySpark.

## How was this patch tested?

New unit tests to `tests.py` & existing unit tests.

Author: Holden Karau <holden@us.ibm.com>

Closes #16211 from holdenk/SPARK-18576-pyspark-taskcontext.
2016-12-20 15:51:21 -08:00
Liang-Chi Hsieh 95c95b71ed [SPARK-18281] [SQL] [PYSPARK] Remove timeout for reading data through socket for local iterator
## What changes were proposed in this pull request?

There is a timeout failure when using `rdd.toLocalIterator()` or `df.toLocalIterator()` for a PySpark RDD and DataFrame:

    df = spark.createDataFrame([[1],[2],[3]])
    it = df.toLocalIterator()
    row = next(it)

    df2 = df.repartition(1000)  # create many empty partitions which increase materialization time so causing timeout
    it2 = df2.toLocalIterator()
    row = next(it2)

The cause of this issue is, we open a socket to serve the data from JVM side. We set timeout for connection and reading through the socket in Python side. In Python we use a generator to read the data, so we only begin to connect the socket once we start to ask data from it. If we don't consume it immediately, there is connection timeout.

In the other side, the materialization time for RDD partitions is unpredictable. So we can't set a timeout for reading data through the socket. Otherwise, it is very possibly to fail.

## How was this patch tested?

Added tests into PySpark.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16263 from viirya/fix-pyspark-localiterator.
2016-12-20 13:12:16 -08:00
Andrew Ray 3c68944b22 [SPARK-16589] [PYTHON] Chained cartesian produces incorrect number of records
## What changes were proposed in this pull request?

Fixes a bug in the python implementation of rdd cartesian product related to batching that showed up in repeated cartesian products with seemingly random results. The root cause being multiple iterators pulling from the same stream in the wrong order because of logic that ignored batching.

`CartesianDeserializer` and `PairDeserializer` were changed to implement `_load_stream_without_unbatching` and borrow the one line implementation of `load_stream` from `BatchedSerializer`. The default implementation of `_load_stream_without_unbatching` was changed to give consistent results (always an iterable) so that it could be used without additional checks.

`PairDeserializer` no longer extends `CartesianDeserializer` as it was not really proper. If wanted a new common super class could be added.

Both `CartesianDeserializer` and `PairDeserializer` now only extend `Serializer` (which has no `dump_stream` implementation) since they are only meant for *de*serialization.

## How was this patch tested?

Additional unit tests (sourced from #14248) plus one for testing a cartesian with zip.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #16121 from aray/fix-cartesian.
2016-12-08 11:08:12 -08:00
Gabriel Huang 70176871ae [SPARK-18361][PYSPARK] Expose RDD localCheckpoint in PySpark
## What changes were proposed in this pull request?

Expose RDD's localCheckpoint() and associated functions in PySpark.

## How was this patch tested?

I added a UnitTest in python/pyspark/tests.py which passes.

I certify that this is my original work, and I license it to the project under the project's open source license.

Gabriel HUANG
Developer at Cardabel (http://cardabel.com/)

Author: Gabriel Huang <gabi.xiaohuang@gmail.com>

Closes #15811 from gabrielhuang/pyspark-localcheckpoint.
2016-11-21 16:08:34 -05:00
Liang-Chi Hsieh 07508bd01d [SPARK-17817][PYSPARK] PySpark RDD Repartitioning Results in Highly Skewed Partition Sizes
## What changes were proposed in this pull request?

Quoted from JIRA description:

Calling repartition on a PySpark RDD to increase the number of partitions results in highly skewed partition sizes, with most having 0 rows. The repartition method should evenly spread out the rows across the partitions, and this behavior is correctly seen on the Scala side.

Please reference the following code for a reproducible example of this issue:

    num_partitions = 20000
    a = sc.parallelize(range(int(1e6)), 2)  # start with 2 even partitions
    l = a.repartition(num_partitions).glom().map(len).collect()  # get length of each partition
    min(l), max(l), sum(l)/len(l), len(l)  # skewed!

In Scala's `repartition` code, we will distribute elements evenly across output partitions. However, the RDD from Python is serialized as a single binary data, so the distribution fails. We need to convert the RDD in Python to java object before repartitioning.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15389 from viirya/pyspark-rdd-repartition.
2016-10-11 11:43:24 -07:00
Yanbo Liang d3b8869763 [SPARK-17585][PYSPARK][CORE] PySpark SparkContext.addFile supports adding files recursively
## What changes were proposed in this pull request?
Users would like to add a directory as dependency in some cases, they can use ```SparkContext.addFile``` with argument ```recursive=true``` to recursively add all files under the directory by using Scala. But Python users can only add file not directory, we should also make it supported.

## How was this patch tested?
Unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15140 from yanboliang/spark-17585.
2016-09-21 01:37:03 -07:00
Yin Huai 0923c4f567 [SPARK-16224] [SQL] [PYSPARK] SparkSession builder's configs need to be set to the existing Scala SparkContext's SparkConf
## What changes were proposed in this pull request?
When we create a SparkSession at the Python side, it is possible that a SparkContext has been created. For this case, we need to set configs of the SparkSession builder to the Scala SparkContext's SparkConf (we need to do so because conf changes on a active Python SparkContext will not be propagated to the JVM side). Otherwise, we may create a wrong SparkSession (e.g. Hive support is not enabled even if enableHiveSupport is called).

## How was this patch tested?
New tests and manual tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #13931 from yhuai/SPARK-16224.
2016-06-28 07:54:44 -07:00
Liang-Chi Hsieh 695d9a0fd4 [SPARK-15433] [PYSPARK] PySpark core test should not use SerDe from PythonMLLibAPI
## What changes were proposed in this pull request?

Currently PySpark core test uses the `SerDe` from `PythonMLLibAPI` which includes many MLlib things. It should use `SerDeUtil` instead.

## How was this patch tested?
Existing tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #13214 from viirya/pycore-use-serdeutil.
2016-05-24 10:10:41 -07:00
Holden Karau 00288ea2a4 [SPARK-13687][PYTHON] Cleanup PySpark parallelize temporary files
## What changes were proposed in this pull request?

Eagerly cleanup PySpark's temporary parallelize cleanup files rather than waiting for shut down.

## How was this patch tested?

Unit tests

Author: Holden Karau <holden@us.ibm.com>

Closes #12233 from holdenk/SPARK-13687-cleanup-pyspark-temporary-files.
2016-04-10 02:34:54 +01:00
Davies Liu 90ca184486 [SPARK-14418][PYSPARK] fix unpersist of Broadcast in Python
## What changes were proposed in this pull request?

Currently, Broaccast.unpersist() will remove the file of broadcast, which should be the behavior of destroy().

This PR added destroy() for Broadcast in Python, to match the sematics in Scala.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #12189 from davies/py_unpersist.
2016-04-06 10:46:34 -07:00
Yong Tang 7db56244fa [SPARK-14368][PYSPARK] Support python.spark.worker.memory with upper-case unit.
## What changes were proposed in this pull request?

This fix tries to address the issue in PySpark where `spark.python.worker.memory`
could only be configured with a lower case unit (`k`, `m`, `g`, `t`). This fix
allows the upper case unit (`K`, `M`, `G`, `T`) to be used as well. This is to
conform to the JVM memory string as is specified in the documentation .

## How was this patch tested?

This fix adds additional test to cover the changes.

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #12163 from yongtang/SPARK-14368.
2016-04-05 12:19:20 +09:00
Shixiong Zhu ee913e6e2d [SPARK-13697] [PYSPARK] Fix the missing module name of TransformFunctionSerializer.loads
## What changes were proposed in this pull request?

Set the function's module name to `__main__` if it's missing in `TransformFunctionSerializer.loads`.

## How was this patch tested?

Manually test in the shell.

Before this patch:
```
>>> from pyspark.streaming import StreamingContext
>>> from pyspark.streaming.util import TransformFunction
>>> ssc = StreamingContext(sc, 1)
>>> func = TransformFunction(sc, lambda x: x, sc.serializer)
>>> func.rdd_wrapper(lambda x: x)
TransformFunction(<function <lambda> at 0x106ac8b18>)
>>> bytes = bytearray(ssc._transformerSerializer.serializer.dumps((func.func, func.rdd_wrap_func, func.deserializers)))
>>> func2 = ssc._transformerSerializer.loads(bytes)
>>> print(func2.func.__module__)
None
>>> print(func2.rdd_wrap_func.__module__)
None
>>>
```
After this patch:
```
>>> from pyspark.streaming import StreamingContext
>>> from pyspark.streaming.util import TransformFunction
>>> ssc = StreamingContext(sc, 1)
>>> func = TransformFunction(sc, lambda x: x, sc.serializer)
>>> func.rdd_wrapper(lambda x: x)
TransformFunction(<function <lambda> at 0x108bf1b90>)
>>> bytes = bytearray(ssc._transformerSerializer.serializer.dumps((func.func, func.rdd_wrap_func, func.deserializers)))
>>> func2 = ssc._transformerSerializer.loads(bytes)
>>> print(func2.func.__module__)
__main__
>>> print(func2.rdd_wrap_func.__module__)
__main__
>>>
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11535 from zsxwing/loads-module.
2016-03-06 08:57:01 -08:00
Gábor Lipták 9bb35c5b59 [SPARK-11295][PYSPARK] Add packages to JUnit output for Python tests
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.
2016-01-20 11:11:10 -08:00
Xiangrui Meng beda901422 Revert "[SPARK-11295] Add packages to JUnit output for Python tests"
This reverts commit c6f971b4ae.
2016-01-19 16:51:17 -08:00
Gábor Lipták c6f971b4ae [SPARK-11295] Add packages to JUnit output for Python tests
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.
2016-01-19 14:06:53 -08:00
Gábor Lipták 163d53e829 [SPARK-7021] Add JUnit output for Python unit tests
WIP

Author: Gábor Lipták <gliptak@gmail.com>

Closes #8323 from gliptak/SPARK-7021.
2015-10-22 15:27:11 -07:00
Davies Liu 232d7f8d42 [SPARK-11114][PYSPARK] add getOrCreate for SparkContext/SQLContext in Python
Also added SQLContext.newSession()

Author: Davies Liu <davies@databricks.com>

Closes #9122 from davies/py_create.
2015-10-19 16:18:20 -07:00
Erik Shilts 7d399c9daa [SPARK-6919] [PYSPARK] Add asDict method to StatCounter
Add method to easily convert a StatCounter instance into a Python dict

https://issues.apache.org/jira/browse/SPARK-6919

Note: This is my original work and the existing Spark license applies.

Author: Erik Shilts <erik.shilts@opower.com>

Closes #5516 from eshilts/statcounter-asdict.
2015-09-29 13:38:15 -07:00
Josh Rosen 2117eea71e [SPARK-10710] Remove ability to disable spilling in core and SQL
It does not make much sense to set `spark.shuffle.spill` or `spark.sql.planner.externalSort` to false: I believe that these configurations were initially added as "escape hatches" to guard against bugs in the external operators, but these operators are now mature and well-tested. In addition, these configurations are not handled in a consistent way anymore: SQL's Tungsten codepath ignores these configurations and will continue to use spilling operators. Similarly, Spark Core's `tungsten-sort` shuffle manager does not respect `spark.shuffle.spill=false`.

This pull request removes these configurations, adds warnings at the appropriate places, and deletes a large amount of code which was only used in code paths that did not support spilling.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8831 from JoshRosen/remove-ability-to-disable-spilling.
2015-09-19 21:40:21 -07:00
Davies Liu 5520418100 [SPARK-10542] [PYSPARK] fix serialize namedtuple
Author: Davies Liu <davies@databricks.com>

Closes #8707 from davies/fix_namedtuple.
2015-09-14 19:46:34 -07:00
Matei Zaharia fe26584a1f [SPARK-9244] Increase some memory defaults
There are a few memory limits that people hit often and that we could
make higher, especially now that memory sizes have grown.

- spark.akka.frameSize: This defaults at 10 but is often hit for map
  output statuses in large shuffles. This memory is not fully allocated
  up-front, so we can just make this larger and still not affect jobs
  that never sent a status that large. We increase it to 128.

- spark.executor.memory: Defaults at 512m, which is really small. We
  increase it to 1g.

Author: Matei Zaharia <matei@databricks.com>

Closes #7586 from mateiz/configs and squashes the following commits:

ce0038a [Matei Zaharia] [SPARK-9244] Increase some memory defaults
2015-07-22 15:28:09 -07:00
Nicholas Hwang a803ac3e06 [SPARK-9021] [PYSPARK] Change RDD.aggregate() to do reduce(mapPartitions()) instead of mapPartitions.fold()
I'm relatively new to Spark and functional programming, so forgive me if this pull request is just a result of my misunderstanding of how Spark should be used.

Currently, if one happens to use a mutable object as `zeroValue` for `RDD.aggregate()`, possibly unexpected behavior can occur.

This is because pyspark's current implementation of `RDD.aggregate()` does not serialize or make a copy of `zeroValue` before handing it off to `RDD.mapPartitions(...).fold(...)`. This results in a single reference to `zeroValue` being used for both `RDD.mapPartitions()` and `RDD.fold()` on each partition. This can result in strange accumulator values being fed into each partition's call to `RDD.fold()`, as the `zeroValue` may have been changed in-place during the `RDD.mapPartitions()` call.

As an illustrative example, submit the following to `spark-submit`:
```
from pyspark import SparkConf, SparkContext
import collections

def updateCounter(acc, val):
    print 'update acc:', acc
    print 'update val:', val
    acc[val] += 1
    return acc

def comboCounter(acc1, acc2):
    print 'combo acc1:', acc1
    print 'combo acc2:', acc2
    acc1.update(acc2)
    return acc1

def main():
    conf = SparkConf().setMaster("local").setAppName("Aggregate with Counter")
    sc = SparkContext(conf = conf)

    print '======= AGGREGATING with ONE PARTITION ======='
    print sc.parallelize(range(1,10), 1).aggregate(collections.Counter(), updateCounter, comboCounter)

    print '======= AGGREGATING with TWO PARTITIONS ======='
    print sc.parallelize(range(1,10), 2).aggregate(collections.Counter(), updateCounter, comboCounter)

if __name__ == "__main__":
    main()
```

One probably expects this to output the following:
```
Counter({1: 1, 2: 1, 3: 1, 4: 1, 5: 1, 6: 1, 7: 1, 8: 1, 9: 1})
```

But it instead outputs this (regardless of the number of partitions):
```
Counter({1: 2, 2: 2, 3: 2, 4: 2, 5: 2, 6: 2, 7: 2, 8: 2, 9: 2})
```

This is because (I believe) `zeroValue` gets passed correctly to each partition, but after `RDD.mapPartitions()` completes, the `zeroValue` object has been updated and is then passed to `RDD.fold()`, which results in all items being double-counted within each partition before being finally reduced at the calling node.

I realize that this type of calculation is typically done by `RDD.mapPartitions(...).reduceByKey(...)`, but hopefully this illustrates some potentially confusing behavior. I also noticed that other `RDD` methods use this `deepcopy` approach to creating unique copies of `zeroValue` (i.e., `RDD.aggregateByKey()` and `RDD.foldByKey()`), and that the Scala implementations do seem to serialize the `zeroValue` object appropriately to prevent this type of behavior.

Author: Nicholas Hwang <moogling@gmail.com>

Closes #7378 from njhwang/master and squashes the following commits:

659bb27 [Nicholas Hwang] Fixed RDD.aggregate() to perform a reduce operation on collected mapPartitions results, similar to how fold currently is implemented. This prevents an initial combOp being performed on each partition with zeroValue (which leads to unexpected behavior if zeroValue is a mutable object) before being combOp'ed with other partition results.
8d8d694 [Nicholas Hwang] Changed dict construction to be compatible with Python 2.6 (cannot use list comprehensions to make dicts)
56eb2ab [Nicholas Hwang] Fixed whitespace after colon to conform with PEP8
391de4a [Nicholas Hwang] Removed used of collections.Counter from RDD tests for Python 2.6 compatibility; used defaultdict(int) instead. Merged treeAggregate test with mutable zero value into aggregate test to reduce code duplication.
2fa4e4b [Nicholas Hwang] Merge branch 'master' of https://github.com/njhwang/spark
ba528bd [Nicholas Hwang] Updated comments regarding protection of zeroValue from mutation in RDD.aggregate(). Added regression tests for aggregate(), fold(), aggregateByKey(), foldByKey(), and treeAggregate(), all with both 1 and 2 partition RDDs. Confirmed that aggregate() is the only problematic implementation as of commit 257236c3e1. Also replaced some parallelizations of ranges with xranges, per the documentation's recommendations of preferring xrange over range.
7820391 [Nicholas Hwang] Updated comments regarding protection of zeroValue from mutation in RDD.aggregate(). Added regression tests for aggregate(), fold(), aggregateByKey(), foldByKey(), and treeAggregate(), all with both 1 and 2 partition RDDs. Confirmed that aggregate() is the only problematic implementation as of commit 257236c3e1.
90d1544 [Nicholas Hwang] Made sure RDD.aggregate() makes a deepcopy of zeroValue for all partitions; this ensures that the mapPartitions call works with unique copies of zeroValue in each partition, and prevents a single reference to zeroValue being used for both map and fold calls on each partition (resulting in possibly unexpected behavior).
2015-07-19 10:30:28 -07:00
MechCoder 20bb10f864 [SPARK-8706] [PYSPARK] [PROJECT INFRA] Add pylint checks to PySpark
This adds Pylint checks to PySpark.

For now this lazy installs using easy_install to /dev/pylint (similar to the pep8 script).
We still need to figure out what rules to be allowed.

Author: MechCoder <manojkumarsivaraj334@gmail.com>

Closes #7241 from MechCoder/pylint and squashes the following commits:

2fc7291 [MechCoder] Remove pylint test fail
6d883a2 [MechCoder] Silence warnings and make pylint tests fail to check if it works in jenkins
f3a5e17 [MechCoder] undefined-variable
ca8b749 [MechCoder] Minor changes
71629f8 [MechCoder] remove trailing whitespace
8498ff9 [MechCoder] Remove blacklisted arguments and pointless statements check
1dbd094 [MechCoder] Disable all checks for now
8b8aa8a [MechCoder] Add pylint configuration file
7871bb1 [MechCoder] [SPARK-8706] [PySpark] [Project infra] Add pylint checks to PySpark
2015-07-15 08:25:53 -07:00
Scott Taylor 6e1c7e2798 [SPARK-7735] [PYSPARK] Raise Exception on non-zero exit from pipe commands
This will allow problems with piped commands to be detected.
This will also allow tasks to be retried where errors are rare (such as network problems in piped commands).

Author: Scott Taylor <github@megatron.me.uk>

Closes #6262 from megatron-me-uk/patch-2 and squashes the following commits:

04ae1d5 [Scott Taylor] Remove spurious empty line
98fa101 [Scott Taylor] fix blank line style error
574b564 [Scott Taylor] Merge pull request #2 from megatron-me-uk/patch-4
0c1e762 [Scott Taylor] Update rdd pipe method for checkCode
ab9a2e1 [Scott Taylor] Update rdd pipe tests for checkCode
eb4801c [Scott Taylor] fix fail_condition
b0ac3a4 [Scott Taylor] Merge pull request #1 from megatron-me-uk/megatron-me-uk-patch-1
a307d13 [Scott Taylor] update rdd tests to test pipe modes
34fcdc3 [Scott Taylor] add optional argument 'mode' for rdd.pipe
a0c0161 [Scott Taylor] fix generator issue
8a9ef9c [Scott Taylor] make check_return_code an iterator
0486ae3 [Scott Taylor] style fixes
8ed89a6 [Scott Taylor] Chain generators to prevent potential deadlock
4153b02 [Scott Taylor] fix list.sort returns None
491d3fc [Scott Taylor] Pass a function handle to assertRaises
3344a21 [Scott Taylor] wrap assertRaises with QuietTest
3ab8c7a [Scott Taylor] remove whitespace for style
cc1a73d [Scott Taylor] fix style issues in pipe test
8db4073 [Scott Taylor] Add a test for rdd pipe functions
1b3dc4e [Scott Taylor] fix missing space around operator style
0974f98 [Scott Taylor] add space between words in multiline string
45f4977 [Scott Taylor] fix line too long style error
5745d85 [Scott Taylor] Remove space to fix style
f552d49 [Scott Taylor] Catch non-zero exit from pipe commands
2015-07-10 19:29:32 -07:00
Josh Rosen 40648c56cd [SPARK-8583] [SPARK-5482] [BUILD] Refactor python/run-tests to integrate 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.
2015-06-27 20:24:34 -07:00
Davies Liu 9b20027222 [SPARK-8202] [PYSPARK] fix infinite loop during external sort in PySpark
The batch size during external sort will grow up to max 10000, then shrink down to zero, causing infinite loop.
Given the assumption that the items usually have similar size, so we don't need to adjust the batch size after first spill.

cc JoshRosen rxin angelini

Author: Davies Liu <davies@databricks.com>

Closes #6714 from davies/batch_size and squashes the following commits:

b170dfb [Davies Liu] update test
b9be832 [Davies Liu] Merge branch 'batch_size' of github.com:davies/spark into batch_size
6ade745 [Davies Liu] update test
5c21777 [Davies Liu] Update shuffle.py
e746aec [Davies Liu] fix batch size during sort
2015-06-18 13:45:58 -07:00
zsxwing 0fc4b96f3e [SPARK-8373] [PYSPARK] Add emptyRDD to pyspark and fix the issue when calling sum on an empty RDD
This PR fixes the sum issue and also adds `emptyRDD` so that it's easy to create a test case.

Author: zsxwing <zsxwing@gmail.com>

Closes #6826 from zsxwing/python-emptyRDD and squashes the following commits:

b36993f [zsxwing] Update the return type to JavaRDD[T]
71df047 [zsxwing] Add emptyRDD to pyspark and fix the issue when calling sum on an empty RDD
2015-06-17 13:59:39 -07:00
Holden Karau 6b18cdc1b1 [SPARK-7711] Add a startTime property to match the corresponding one in Scala
Author: Holden Karau <holden@pigscanfly.ca>

Closes #6275 from holdenk/SPARK-771-startTime-is-missing-from-pyspark and squashes the following commits:

06662dc [Holden Karau] add mising blank line for style checks
7a87410 [Holden Karau] add back missing newline
7a7876b [Holden Karau] Add a startTime property to match the corresponding one in the Scala SparkContext
2015-05-21 14:08:57 -07:00
Daoyuan Wang c2437de189 [SPARK-7150] SparkContext.range() and SQLContext.range()
This PR is based on #6081, thanks adrian-wang.

Closes #6081

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #6230 from davies/range and squashes the following commits:

d3ce5fe [Davies Liu] add tests
789eda5 [Davies Liu] add range() in Python
4590208 [Davies Liu] Merge commit 'refs/pull/6081/head' of github.com:apache/spark into range
cbf5200 [Daoyuan Wang] let's add python support in a separate PR
f45e3b2 [Daoyuan Wang] remove redundant toLong
617da76 [Daoyuan Wang] fix safe marge for corner cases
867c417 [Daoyuan Wang] fix
13dbe84 [Daoyuan Wang] update
bd998ba [Daoyuan Wang] update comments
d3a0c1b [Daoyuan Wang] add range api()
2015-05-18 21:43:12 -07:00
Davies Liu 32fbd297dd [SPARK-6216] [PYSPARK] check python version of worker with driver
This PR revert #5404, change to pass the version of python in driver into JVM, check it in worker before deserializing closure, then it can works with different major version of Python.

Author: Davies Liu <davies@databricks.com>

Closes #6203 from davies/py_version and squashes the following commits:

b8fb76e [Davies Liu] fix test
6ce5096 [Davies Liu] use string for version
47c6278 [Davies Liu] check python version of worker with driver
2015-05-18 12:55:13 -07:00
Vinod K C dda6d9f404 [SPARK-7438] [SPARK CORE] Fixed validation of relativeSD in countApproxDistinct
Author: Vinod K C <vinod.kc@huawei.com>

Closes #5974 from vinodkc/fix_countApproxDistinct_Validation and squashes the following commits:

3a3d59c [Vinod K C] Reverted removal of validation relativeSD<0.000017
799976e [Vinod K C] Removed testcase to assert IAE when relativeSD>3.7
8ddbfae [Vinod K C] Remove blank line
b1b00a3 [Vinod K C] Removed relativeSD validation from python API,RDD.scala will do validation
122d378 [Vinod K C] Fixed validation of relativeSD in  countApproxDistinct
2015-05-09 10:03:15 +01:00